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/06/16 22:00:29 UTC

[GitHub] [kafka] jeffkbkim opened a new pull request, #13870: KAFKA-14500; [5/N]

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

   *More detailed description of your change,
   if necessary. The PR title and PR message become
   the squashed commit message, so use a separate
   comment to ping reviewers.*
   
   *Summary of testing strategy (including rationale)
   for the feature or bug fix. Unit and/or integration
   tests are expected for any behaviour change and
   system tests should be considered for larger changes.*
   
   ### 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] dajac commented on a diff in pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }
+
+                result = completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member, group.rebalanceTimeoutMs());
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Void>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if ((member.assignment() == null || member.assignment().length == 0) && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        if (group.isInState(EMPTY) || group.isInState(DEAD)) {
+            // Always accept the request when the group is empty or dead
+            return true;
+
+        } else if (group.isInState(PREPARING_REBALANCE)) {
+            // An existing member is accepted if it is already awaiting. New members are accepted
+            // up to the max group size. Note that the number of awaiting members is used here
+            // for two reasons:
+            // 1) the group size is not reliable as it could already be above the max group size
+            //    if the max group size was reduced.
+            // 2) using the number of awaiting members allows to kick out the last rejoining
+            //    members of the group.
+            return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                group.numAwaitingJoinResponse() < genericGroupMaxSize;
+
+        } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+            // An existing member is accepted. New members are accepted up to the max group size.
+            // Note that the group size is used here. When the group transitions to CompletingRebalance,
+            // members who haven't rejoined are removed.
+            return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+        } else {
+            throw new IllegalStateException("Cannot handle state " + group.stateAsString());
+        }
+    }
+
+    /**
+     * Update a static member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group of the static member.
+     * @param oldMemberId     The existing static member id.
+     * @param newMemberId     The new joining static member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateStaticMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String oldMemberId,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String currentLeader = group.leaderOrNull();
+        GenericGroupMember member = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+
+        // Heartbeat of old member id will expire without effect since the group no longer contains that member id.
+        // New heartbeat shall be scheduled with new member id.
+        rescheduleGenericGroupMemberHeartbeat(group, member);
+
+        int oldRebalanceTimeoutMs = member.rebalanceTimeoutMs();
+        int oldSessionTimeoutMs = member.sessionTimeoutMs();
+        JoinGroupRequestProtocolCollection oldProtocols = member.supportedProtocols();
+
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        if (group.isInState(STABLE)) {
+            // Check if group's selected protocol of next generation will change, if not, simply store group to persist
+            // the updated static member, if yes, rebalance should be triggered to keep the group's assignment
+            // and selected protocol consistent
+            String groupInstanceId = request.groupInstanceId();
+            String selectedProtocolForNextGeneration = group.selectProtocol();
+            if (group.protocolName().orElse("").equals(selectedProtocolForNextGeneration)) {
+                log.info("Static member which joins during Stable stage and doesn't affect " +
+                    "the selected protocol will not trigger a rebalance.");
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    // Starting from version 9 of the JoinGroup API, static members are able to
+                    // skip running the assignor based on the `SkipAssignment` field. We leverage
+                    // this to tell the leader that it is the leader of the group but by skipping
+                    // running the assignor while the group is in stable state.
+                    // Notes:
+                    // 1) This allows the leader to continue monitoring metadata changes for the
+                    // group. Note that any metadata changes happening while the static leader is
+                    // down won't be noticed.
+                    // 2) The assignors are not idempotent nor free from side effects. This is why
+                    // we skip entirely the assignment step as it could generate a different group
+                    // assignment which would be ignored by the group coordinator because the group
+                    // is the stable state.
+                    boolean supportSkippingAssignment = context.apiVersion() >= 9;
+
+                    if (t != null) {
+                        log.warn("Failed to persist metadata for group {}: {}", group.groupId(), t.getMessage());
+
+                        // Failed to persist the member id of the given static member, revert the update of the static member in the group.
+                        group.updateMember(member, oldProtocols, oldRebalanceTimeoutMs, oldSessionTimeoutMs, null);
+                        GenericGroupMember oldMember = group.replaceStaticMember(groupInstanceId, newMemberId, oldMemberId);
+                        rescheduleGenericGroupMemberHeartbeat(group, oldMember);
+
+                        short errorCode = Errors.forException(t).code();
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setMembers(Collections.emptyList())
+                                .setMemberId(UNKNOWN_MEMBER_ID)
+                                .setGenerationId(group.generationId())
+                                .setProtocolName(group.protocolName().orElse(null))
+                                .setProtocolType(group.protocolType().orElse(null))
+                                .setLeader(currentLeader)
+                                .setSkipAssignment(false)
+                                .setErrorCode(errorCode)
+                        );
+
+                    } else if (supportSkippingAssignment) {
+                        boolean isLeader = group.isLeader(newMemberId);
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (isLeader) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        group.completeJoinFuture(member, new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(isLeader)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+
+                    } else {
+                        // Prior to version 9 of the JoinGroup API, we wanted to avoid current leader
+                        // performing trivial assignment while the group is in stable stage, because
+                        // the new assignment in leader's next sync call won't be broadcast by a stable group.
+                        // This could be guaranteed by always returning the old leader id so that the current
+                        // leader won't assume itself as a leader based on the returned message, since the new
+                        // member.id won't match returned leader id, therefore no assignment will be performed.
+
+                        group.completeJoinFuture(member, new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(currentLeader)
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                return maybePrepareRebalance(
+                    group,
+                    "Group's selectedProtocol will change because static member " +
+                        member.memberId() + " with instance id " + groupInstanceId +
+                        " joined with change of protocol; " + "client reason: " + joinReason
+                );
+            }
+
+        } else if (group.isInState(COMPLETING_REBALANCE)) {
+            // if the group is in after-sync stage, upon getting a new join-group of a known static member
+            // we should still trigger a new rebalance, since the old member may already be sent to the leader
+            // for assignment, and hence when the assignment gets back there would be a mismatch of the old member id
+            // with the new replaced member id. As a result the new member id would not get any assignment.
+            return prepareRebalance(group,
+                "Updating metadata for static member " + member.memberId() + " with instance id " +
+                    request.groupInstanceId() + "; client reason: " + joinReason);
+
+        } else if (group.isInState(EMPTY) || group.isInState(DEAD)) {
+            throw new IllegalStateException("Group " + group.groupId() + " was not supposed to be in the state " +
+                group.stateAsString() + " when the unknown static member " + request.groupInstanceId() + " rejoins.");

Review Comment:
   What does happen when this exception is thrown? I mean, where is it handled?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }
+
+                result = completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member, group.rebalanceTimeoutMs());
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Void>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if ((member.assignment() == null || member.assignment().length == 0) && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        if (group.isInState(EMPTY) || group.isInState(DEAD)) {
+            // Always accept the request when the group is empty or dead
+            return true;
+
+        } else if (group.isInState(PREPARING_REBALANCE)) {
+            // An existing member is accepted if it is already awaiting. New members are accepted
+            // up to the max group size. Note that the number of awaiting members is used here
+            // for two reasons:
+            // 1) the group size is not reliable as it could already be above the max group size
+            //    if the max group size was reduced.
+            // 2) using the number of awaiting members allows to kick out the last rejoining
+            //    members of the group.
+            return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                group.numAwaitingJoinResponse() < genericGroupMaxSize;
+
+        } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+            // An existing member is accepted. New members are accepted up to the max group size.
+            // Note that the group size is used here. When the group transitions to CompletingRebalance,
+            // members who haven't rejoined are removed.
+            return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+        } else {
+            throw new IllegalStateException("Cannot handle state " + group.stateAsString());
+        }
+    }
+
+    /**
+     * Update a static member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group of the static member.
+     * @param oldMemberId     The existing static member id.
+     * @param newMemberId     The new joining static member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateStaticMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String oldMemberId,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String currentLeader = group.leaderOrNull();
+        GenericGroupMember member = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+
+        // Heartbeat of old member id will expire without effect since the group no longer contains that member id.
+        // New heartbeat shall be scheduled with new member id.
+        rescheduleGenericGroupMemberHeartbeat(group, member);
+
+        int oldRebalanceTimeoutMs = member.rebalanceTimeoutMs();
+        int oldSessionTimeoutMs = member.sessionTimeoutMs();
+        JoinGroupRequestProtocolCollection oldProtocols = member.supportedProtocols();
+
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        if (group.isInState(STABLE)) {
+            // Check if group's selected protocol of next generation will change, if not, simply store group to persist
+            // the updated static member, if yes, rebalance should be triggered to keep the group's assignment
+            // and selected protocol consistent
+            String groupInstanceId = request.groupInstanceId();
+            String selectedProtocolForNextGeneration = group.selectProtocol();
+            if (group.protocolName().orElse("").equals(selectedProtocolForNextGeneration)) {
+                log.info("Static member which joins during Stable stage and doesn't affect " +
+                    "the selected protocol will not trigger a rebalance.");
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {

Review Comment:
   I think that the non-error case is actually incorrect based on the implementation in the runtime. The issue is that the future will be completed immediately after the records are written. This means that we would send the response before the record is committed. I think that the future should be completed only when the records are committed.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }

Review Comment:
   Actually, it seems that `completeGenericGroupJoin` is already called in a few places on those paths.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata())).collect(Collectors.toList());
+
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(protocols))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(protocols)) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(protocols)) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            return tryCompleteJoinElseSchedule(group);
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Errors> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((error, t) -> {
+                    if (t != null || error != Errors.NONE) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        String message = t != null ? t.getMessage() : error.message();
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), message);
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member);
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {

Review Comment:
   correct. however i think that the fundamental issues is that we do not cancel the session timeout while doing a rebalance. this is why we have this condition here. if we fix this, we may be able to remove it.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }

Review Comment:
   This feels a bit like a hack. I was wondering if we could push that call to `completeGenericGroupJoin(group)` into the `genericGroupJoinNewMember` and `genericGroupJoinExistingMember` paths instead of handling it here for all cases. Is it something that you have considered?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -179,62 +278,161 @@ GroupMetadataManager build() {
     private final int consumerGroupHeartbeatIntervalMs;
 
     /**
-     * The topics metadata (or image).
+     * The metadata image.
+     */
+    private MetadataImage metadataImage;
+
+    // Rest of the fields are used for the generic group APIs.

Review Comment:
   nit: Let's remove this one as well.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java:
##########
@@ -417,6 +417,27 @@ public static Record newGroupMetadataRecord(
         );
     }
 
+    /**
+     * Used only for testing.
+     */
+    public static Record newGroupMetadataRecord(

Review Comment:
   nit: Could we move this to the test then?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }
+
+                result = completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))

Review Comment:
   nit: Should we have an overload of `supportsProtocols` which takes `request.protocols()`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }
+
+                result = completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member, group.rebalanceTimeoutMs());
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Void>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if ((member.assignment() == null || member.assignment().length == 0) && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        if (group.isInState(EMPTY) || group.isInState(DEAD)) {

Review Comment:
   Did you consider using a switch here? It seems that it would fit nicely.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }
+
+                result = completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member, group.rebalanceTimeoutMs());
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Void>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+

Review Comment:
   nit: extra empty line.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }
+
+                result = completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member, group.rebalanceTimeoutMs());
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Void>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;

Review Comment:
   It seems that the condition was `group.is(Empty)` in Scala. What's the reason for changing it?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata())).collect(Collectors.toList());
+
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(protocols))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(protocols)) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(protocols)) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            return tryCompleteJoinElseSchedule(group);
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Errors> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((error, t) -> {
+                    if (t != null || error != Errors.NONE) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        String message = t != null ? t.getMessage() : error.message();
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), message);
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member);

Review Comment:
   This is correct. btw, if we remove it, I think that we need to ensure that the session timeout is cancelled when a member rejoins.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }
+
+                result = completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member, group.rebalanceTimeoutMs());
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Void>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if ((member.assignment() == null || member.assignment().length == 0) && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        if (group.isInState(EMPTY) || group.isInState(DEAD)) {
+            // Always accept the request when the group is empty or dead
+            return true;
+
+        } else if (group.isInState(PREPARING_REBALANCE)) {
+            // An existing member is accepted if it is already awaiting. New members are accepted
+            // up to the max group size. Note that the number of awaiting members is used here
+            // for two reasons:
+            // 1) the group size is not reliable as it could already be above the max group size
+            //    if the max group size was reduced.
+            // 2) using the number of awaiting members allows to kick out the last rejoining
+            //    members of the group.
+            return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                group.numAwaitingJoinResponse() < genericGroupMaxSize;
+
+        } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+            // An existing member is accepted. New members are accepted up to the max group size.
+            // Note that the group size is used here. When the group transitions to CompletingRebalance,
+            // members who haven't rejoined are removed.
+            return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+        } else {
+            throw new IllegalStateException("Cannot handle state " + group.stateAsString());
+        }
+    }
+
+    /**
+     * Update a static member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group of the static member.
+     * @param oldMemberId     The existing static member id.
+     * @param newMemberId     The new joining static member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateStaticMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String oldMemberId,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String currentLeader = group.leaderOrNull();
+        GenericGroupMember member = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+
+        // Heartbeat of old member id will expire without effect since the group no longer contains that member id.
+        // New heartbeat shall be scheduled with new member id.

Review Comment:
   That's correct. However, it may be better to just cancel the previous one.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata())).collect(Collectors.toList());
+
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(protocols))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(protocols)) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(protocols)) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            return tryCompleteJoinElseSchedule(group);
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Errors> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((error, t) -> {
+                    if (t != null || error != Errors.NONE) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        String message = t != null ? t.getMessage() : error.message();
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), message);
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member);
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata()))
+            .collect(Collectors.toList());
+
+        group.updateMember(member, protocols, request.rebalanceTimeoutMs(), request.sessionTimeoutMs(), responseFuture);
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            protocols
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, newMemberJoinTimeoutMs);

Review Comment:
   right. separation of concerns would be the benefit. 



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }
+
+                result = completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member, group.rebalanceTimeoutMs());
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Void>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if ((member.assignment() == null || member.assignment().length == 0) && error == Errors.NONE) {

Review Comment:
   nit: Should we have an method such as `hasAssignment()` in member?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata())).collect(Collectors.toList());
+
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(protocols))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(protocols)) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(protocols)) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            return tryCompleteJoinElseSchedule(group);
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Errors> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((error, t) -> {
+                    if (t != null || error != Errors.NONE) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        String message = t != null ? t.getMessage() : error.message();
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), message);
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member);

Review Comment:
   so that i understand: during completing rebalance, we schedule both pending sync (rebalance timeout) and heartbeats (session timeout). in practice, session timeout << rebalance timeout so heartbeats would expire and remove members. With cooperative rebalancing members should still be able to fetch records during completing rebalance phase. We want to extend the heartbeat here to rebalance timeout so that members are not removed by session timeout.
   
   is this correct?
   
   If so, i agree removing the heartbeat schedule sounds like the best approach since members are removed when pending sync expires anyways.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +295,21 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isValidGroupId(request.groupId(), ApiKeys.forId(request.apiKey()))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleGenericGroupOperation("generic-group-join",
+            topicPartitionFor(request.groupId()),
+            coordinator -> coordinator.genericGroupJoin(context, request, responseFuture));
+
+        return responseFuture;

Review Comment:
   i don't think this is the right place; we need to add the logic inside group metadata manager. i have done so in the sync PR.
   
   the reason is that for generic group apis, the append future is what we want the logic for when handling log append/commit errors.
   
   whereas for the new protocol, the consumer group heartbeat waits to return the results from the append/commit.
   
   



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorResult.java:
##########
@@ -37,18 +38,51 @@ public class CoordinatorResult<T, U> {
      */
     private final T response;
 
+    /**
+     * The future to complete once the records are committed.
+     */
+    private final CompletableFuture<T> appendFuture;

Review Comment:
   this is only used when records are generated (and need to be appended to the log) so i think append future makes more sense.
   
   `completionFuture` will be confusing alongside coordinator event's `future` field. wdyt?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1245,4 +1422,1304 @@ public static String consumerGroupSessionTimeoutKey(String groupId, String membe
     public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) {
         return "revocation-timeout-" + groupId + "-" + memberId;
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            removeGroup(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> genericGroup.add(member, null));
+            groups.put(groupId, genericGroup);
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if a group was newly created, we need to append
+                // records to the log to commit the group to the timeline data structure.
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setErrorCode(appendGroupMetadataErrorToResponseError(Errors.forException(t)).code()));
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newEmptyGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Attempt to complete join group phase. We do not complete
+     * the join group phase if this is the initial rebalance.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinPhase(GenericGroup group) {
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                responseFuture
+            );
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (JoinGroupRequest.requiresKnownMemberId(context.apiVersion())) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String genericGroupHeartbeatKey = genericGroupHeartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                genericGroupHeartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberThenRebalanceOrCompleteJoin(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " + group.stateAsString() +
+                            "; client reason: " + JoinGroupRequest.joinReason(request),
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(group.isLeader(memberId) ?
+                                group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(genericGroupJoinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(genericGroupHeartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                genericGroupJoinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}) with {} members",
+                    groupId, group.generationId(), topicPartition, group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(genericGroupHeartbeatKey(groupId, member.memberId()));

Review Comment:
   yes. 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] jeffkbkim commented on a diff in pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1230,1358 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = groups.get(groupId) == null;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We create records here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()));
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                newGroupResult = new CoordinatorResult<>(records, appendFuture);
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if the group is new, we need to
+                // persist the group to the timeline map.
+                result = newGroupResult;
+            }
+        }
+        return result;
+    }
+
+    private CoordinatorResult<Void, Record> tryCompleteJoinPhase(GenericGroup group) {
+        // Attempt to complete join group phase. We do not complete
+        // the join group phase if this is the initial rebalance.
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(heartbeatKey(groupId, member.memberId()));
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalanceOrCompleteJoin(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalanceOrCompleteJoin(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.previousState() == EMPTY) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalanceOrCompleteJoin(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state. Otherwise, try
+     * to complete the join phase.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybePrepareRebalanceOrCompleteJoin(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        } else {
+            return tryCompleteJoinPhase(group);
+        }
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<Void, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.isInState(EMPTY);
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if (!member.hasAssignment() && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<Void, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        switch (group.currentState()) {
+            case EMPTY:
+            case DEAD:
+                // Always accept the request when the group is empty or dead
+                return true;
+            case PREPARING_REBALANCE:
+                // An existing member is accepted if it is already awaiting. New members are accepted
+                // up to the max group size. Note that the number of awaiting members is used here
+                // for two reasons:
+                // 1) the group size is not reliable as it could already be above the max group size
+                //    if the max group size was reduced.
+                // 2) using the number of awaiting members allows to kick out the last rejoining
+                //    members of the group.
+                return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                    group.numAwaitingJoinResponse() < genericGroupMaxSize;
+            case COMPLETING_REBALANCE:
+            case STABLE:
+                // An existing member is accepted. New members are accepted up to the max group size.
+                // Note that the group size is used here. When the group transitions to CompletingRebalance,
+                // members who haven't rejoined are removed.
+                return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+            default:
+                throw new IllegalStateException("Unknown group state: " + group.stateAsString());
+        }
+    }
+
+    /**
+     * Update a static member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group of the static member.
+     * @param oldMemberId     The existing static member id.
+     * @param newMemberId     The new joining static member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateStaticMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String oldMemberId,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String currentLeader = group.leaderOrNull();
+        GenericGroupMember member = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+
+        // Heartbeat of old member id will expire without effect since the group no longer contains that member id.
+        // New heartbeat shall be scheduled with new member id.
+        rescheduleGenericGroupMemberHeartbeat(group, member);
+
+        int oldRebalanceTimeoutMs = member.rebalanceTimeoutMs();
+        int oldSessionTimeoutMs = member.sessionTimeoutMs();
+        JoinGroupRequestProtocolCollection oldProtocols = member.supportedProtocols();
+
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        if (group.isInState(STABLE)) {
+            // Check if group's selected protocol of next generation will change, if not, simply store group to persist
+            // the updated static member, if yes, rebalance should be triggered to keep the group's assignment
+            // and selected protocol consistent
+            String groupInstanceId = request.groupInstanceId();
+            String selectedProtocolForNextGeneration = group.selectProtocol();
+            if (group.protocolName().orElse("").equals(selectedProtocolForNextGeneration)) {
+                log.info("Static member which joins during Stable stage and doesn't affect " +
+                    "the selected protocol will not trigger a rebalance.");
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    // Starting from version 9 of the JoinGroup API, static members are able to
+                    // skip running the assignor based on the `SkipAssignment` field. We leverage
+                    // this to tell the leader that it is the leader of the group but by skipping
+                    // running the assignor while the group is in stable state.
+                    // Notes:
+                    // 1) This allows the leader to continue monitoring metadata changes for the
+                    // group. Note that any metadata changes happening while the static leader is
+                    // down won't be noticed.
+                    // 2) The assignors are not idempotent nor free from side effects. This is why
+                    // we skip entirely the assignment step as it could generate a different group
+                    // assignment which would be ignored by the group coordinator because the group
+                    // is the stable state.
+                    boolean supportSkippingAssignment = context.apiVersion() >= 9;
+
+                    if (t != null) {
+                        log.warn("Failed to persist metadata for group {}: {}", group.groupId(), t.getMessage());
+
+                        // Failed to persist the member id of the given static member, revert the update of the static member in the group.
+                        group.updateMember(member, oldProtocols, oldRebalanceTimeoutMs, oldSessionTimeoutMs, null);
+                        GenericGroupMember oldMember = group.replaceStaticMember(groupInstanceId, newMemberId, oldMemberId);
+                        rescheduleGenericGroupMemberHeartbeat(group, oldMember);
+
+                        short errorCode = Errors.forException(t).code();
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setMembers(Collections.emptyList())
+                                .setMemberId(UNKNOWN_MEMBER_ID)
+                                .setGenerationId(group.generationId())
+                                .setProtocolName(group.protocolName().orElse(null))
+                                .setProtocolType(group.protocolType().orElse(null))
+                                .setLeader(currentLeader)
+                                .setSkipAssignment(false)
+                                .setErrorCode(errorCode)
+                        );
+
+                    } else if (supportSkippingAssignment) {
+                        boolean isLeader = group.isLeader(newMemberId);
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (isLeader) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        group.completeJoinFuture(member, new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(isLeader)
+                            .setErrorCode(Errors.NONE.code())

Review Comment:
   isn't it more readable to keep 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }

Review Comment:
   for all places i use the old for each / for loops, there is an error
   `Variable used in lambda expression should be final or effectively final`
   because i reuse variables (mainly JoinGroupRequestData & responseFutures). i can use new variables instead, would that be 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] jeffkbkim commented on a diff in pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1348 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();

Review Comment:
   for 1) doesn't it require a bump in the group metadata value version to add the default value?
   
   2) i don't see much value in this and it feels more different to handle it this way compared to other record types in ReplicatedGroupCoordinator#replay(Record)



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1243,4 +1406,1283 @@ public static String consumerGroupSessionTimeoutKey(String groupId, String membe
     public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) {
         return "revocation-timeout-" + groupId + "-" + memberId;
     }
+
+     /** Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if a group was newly created, we need to append
+                // records to the log to commit the group to the timeline data structure.
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newEmptyGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Attempt to complete join group phase. We do not complete
+     * the join group phase if this is the initial rebalance.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinPhase(GenericGroup group) {
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                responseFuture
+            );
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (JoinGroupRequest.requiresKnownMemberId(context.apiVersion())) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberThenRebalanceOrCompleteJoin(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " + group.stateAsString() +
+                            "; client reason: " + JoinGroupRequest.joinReason(request),
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(group.isLeader(memberId) ?
+                                group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}) with {} members",
+                    groupId, group.generationId(), topicPartition, group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(heartbeatKey(groupId, member.memberId()));
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            false,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalanceOrCompleteJoin(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member. Then begin a rebalance or complete the join phase.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateMemberThenRebalanceOrCompleteJoin(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalanceOrCompleteJoin(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.previousState() == EMPTY) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalanceOrCompleteJoin(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + JoinGroupRequest.joinReason(request));
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state. Otherwise, try
+     * to complete the join phase.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybePrepareRebalanceOrCompleteJoin(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        } else {
+            return maybeCompleteJoinPhase(group);
+        }
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     *
+     * Package private for testing.
+     */
+    CoordinatorResult<Void, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.isInState(EMPTY);
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : maybeCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if (!member.hasAssignment() && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            false,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<Void, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {

Review Comment:
   No. I think that the current coordinator triggers a rebalance if the number of members is higher than the max when a group is loaded.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -579,4 +618,32 @@ public void shutdown() {
         Utils.closeQuietly(runtime, "coordinator runtime");
         log.info("Shutdown complete.");
     }
+
+    private static boolean isGroupIdNotEmpty(String groupId) {
+        return groupId != null && !groupId.isEmpty();
+    }
+
+    private static Errors toResponseError(Errors appendError) {
+        switch (appendError) {
+            case UNKNOWN_TOPIC_OR_PARTITION:
+            case NOT_ENOUGH_REPLICAS:
+            case NOT_ENOUGH_REPLICAS_AFTER_APPEND:
+                return COORDINATOR_NOT_AVAILABLE;
+
+            case NOT_LEADER_OR_FOLLOWER:
+            case KAFKA_STORAGE_ERROR:
+                return NOT_COORDINATOR;
+
+            case REQUEST_TIMED_OUT:

Review Comment:
   This request timeout was coming from the delayed produce op in the purgatory. We don't have this anymore.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +282,32 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isGroupIdNotEmpty(request.groupId())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleWriteOperation("generic-group-join",
+            topicPartitionFor(request.groupId()),
+            coordinator -> coordinator.genericGroupJoin(context, request, responseFuture)
+        ).exceptionally(exception -> {
+            log.error("Request {} hit an unexpected exception: {}",
+                request, exception.getMessage());

Review Comment:
   changed to logging only when the response future is not complete



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -301,10 +302,12 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             topicPartitionFor(request.groupId()),
             coordinator -> coordinator.genericGroupJoin(context, request, responseFuture)
         ).exceptionally(exception -> {
-            if (!responseFuture.isDone()) {
+            if (!(exception instanceof KafkaException)) {
                 log.error("Request {} hit an unexpected exception: {}",

Review Comment:
   nit: Could we say `JoinGroup request {} hit....`?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
core/src/main/scala/kafka/server/BrokerServer.scala:
##########
@@ -531,7 +531,12 @@ class BrokerServer(
         config.consumerGroupHeartbeatIntervalMs,
         config.consumerGroupMaxSize,
         config.consumerGroupAssignors,
-        config.offsetsTopicSegmentBytes
+        config.offsetsTopicSegmentBytes,
+        config.groupMaxSize,
+        config.groupInitialRebalanceDelay,
+        5 * 60 * 1000,

Review Comment:
   Should we replace this by a constant if we can't change it based on config?



-- 
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 pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13870:
URL: https://github.com/apache/kafka/pull/13870#issuecomment-1625488949

   There are issues with the build as well. Could you look into 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }
-    }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, new ArrayList<>(group.allMembers()).get(0).memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());

Review Comment:
   related to when a group is not found. have reverted and updated the error code



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }
-    }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);

Review Comment:
   there's a +1 on all advance clocks in scala. i haven't actually looked but assumed that it's in place due to how the purgatory works.
   
   the java timer implementation does not require a +1



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -579,4 +618,32 @@ public void shutdown() {
         Utils.closeQuietly(runtime, "coordinator runtime");
         log.info("Shutdown complete.");
     }
+
+    private static boolean isGroupIdNotEmpty(String groupId) {
+        return groupId != null && !groupId.isEmpty();
+    }
+
+    private static Errors toResponseError(Errors appendError) {

Review Comment:
   confirmed that `storeGroup` and `storeOffsets` have different handling.
   
   this will still be shared amongst join/sync/leave group, so i'll rename this to `appendGroupMetadataErrorToResponseError`, wdyt?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1221,1331 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",

Review Comment:
   is your suggestion to iterate through all groups & members and log each member after loading a partition is complete?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -247,31 +324,78 @@ GroupMetadataManager build() {
      */
     private MetadataImage metadataImage;
 
+    /**
+     * An empty result returned to the state machine. This means that
+     * there are no records to append to the log.
+     *
+     * Package private for testing.
+     */
+    static final CoordinatorResult<Void, Record> EMPTY_RESULT =
+        new CoordinatorResult<>(Collections.emptyList(), CompletableFuture.completedFuture(null));

Review Comment:
   that forces the CoordinatorResult class to become non-generic which i don't think we want.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -171,70 +260,152 @@ GroupMetadataManager build() {
     /**
      * The maximum number of members allowed in a single consumer group.
      */
-    private final int consumerGroupMaxSize;
+    private final int groupMaxSize;
 
     /**
      * The heartbeat interval for consumer groups.
      */
     private final int consumerGroupHeartbeatIntervalMs;
 
     /**
-     * The topics metadata (or image).
+     * The metadata image.
+     */
+    private MetadataImage metadataImage;
+
+    // Rest of the fields are used for the generic group APIs.
+
+    /**
+     * An empty result returned to the state machine. This means that
+     * there are no records to append to the log.
+     *
+     * Package private for testing.
+     */
+    static final CoordinatorResult<CompletableFuture<Errors>, Record> EMPTY_RESULT =
+        new CoordinatorResult<>(Collections.emptyList(), CompletableFuture.completedFuture(null));
+
+    /**
+     * Initial rebalance delay for members joining a generic group.
+     */
+    private final int initialRebalanceDelayMs;

Review Comment:
   nit: Could we prefix all those attributes with `genericGroup`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -171,70 +260,152 @@ GroupMetadataManager build() {
     /**
      * The maximum number of members allowed in a single consumer group.
      */
-    private final int consumerGroupMaxSize;
+    private final int groupMaxSize;

Review Comment:
   We can't use the same config both both the consumer and the generic group because we have two configs for each.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;

Review Comment:
   Should we have a method helper to validate the request?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.

Review Comment:
   I would remove this because it will be outdated extremely quickly.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);

Review Comment:
   When the group is created to the first time, I think that we need to write a record to the log; the group could be reverted in the timeline hash map otherwise.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata())).collect(Collectors.toList());
+
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(protocols))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(protocols)) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(protocols)) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            return tryCompleteJoinElseSchedule(group);
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Errors> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((error, t) -> {
+                    if (t != null || error != Errors.NONE) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        String message = t != null ? t.getMessage() : error.message();
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), message);
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member);

Review Comment:
   We need to discuss whether we want to start the heartbeat timer here or not. See [KAFKA-13766](https://issues.apache.org/jira/browse/KAFKA-13766).



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata())).collect(Collectors.toList());
+
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(protocols))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(protocols)) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(protocols)) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            return tryCompleteJoinElseSchedule(group);
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Errors> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((error, t) -> {
+                    if (t != null || error != Errors.NONE) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        String message = t != null ? t.getMessage() : error.message();
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), message);
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member);
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata()))
+            .collect(Collectors.toList());
+
+        group.updateMember(member, protocols, request.rebalanceTimeoutMs(), request.sessionTimeoutMs(), responseFuture);
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            protocols
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, newMemberJoinTimeoutMs);

Review Comment:
   I wonder if we should use a different timer for this case. Have you considered it?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata())).collect(Collectors.toList());
+
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(protocols))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(protocols)) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(protocols)) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            return tryCompleteJoinElseSchedule(group);
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Errors> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((error, t) -> {
+                    if (t != null || error != Errors.NONE) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        String message = t != null ? t.getMessage() : error.message();
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), message);
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member);
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata()))
+            .collect(Collectors.toList());
+
+        group.updateMember(member, protocols, request.rebalanceTimeoutMs(), request.sessionTimeoutMs(), responseFuture);
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            protocols
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, newMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Errors>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = initialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - initialRebalanceDelayMs, 0);
+            group.setInitialRebalanceTimeoutMs(remainingMs);
+            group.setInitialRebalanceDelayMs(delayMs);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group)
+            );
+        }
+
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group
+    ) {
+        int remainingMs = group.initialRebalanceTimeoutMs();
+        int delayMs = group.initialRebalanceDelayMs();
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(initialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            group.setInitialRebalanceTimeoutMs(newRemainingMs);
+            group.setInitialRebalanceDelayMs(newDelayMs);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group)

Review Comment:
   Instead of storing the initial rebalance timeout and the initial rebalance delay in the group, could we imagine passing them as arguments to `tryCompleteInitialRebalanceElseSchedule`? That could simplify the logic as everything would be self contain here.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -171,70 +260,152 @@ GroupMetadataManager build() {
     /**
      * The maximum number of members allowed in a single consumer group.
      */
-    private final int consumerGroupMaxSize;
+    private final int groupMaxSize;
 
     /**
      * The heartbeat interval for consumer groups.
      */
     private final int consumerGroupHeartbeatIntervalMs;
 
     /**
-     * The topics metadata (or image).
+     * The metadata image.
+     */
+    private MetadataImage metadataImage;
+
+    // Rest of the fields are used for the generic group APIs.
+
+    /**
+     * An empty result returned to the state machine. This means that
+     * there are no records to append to the log.
+     *
+     * Package private for testing.
+     */
+    static final CoordinatorResult<CompletableFuture<Errors>, Record> EMPTY_RESULT =
+        new CoordinatorResult<>(Collections.emptyList(), CompletableFuture.completedFuture(null));
+
+    /**
+     * Initial rebalance delay for members joining a generic group.
+     */
+    private final int initialRebalanceDelayMs;
+
+    /**
+     * The timeout used to wait for a new member in milliseconds.
+     */
+    private final int newMemberJoinTimeoutMs;
+
+    /**
+     * The group minimum session timeout.
+     */
+    private final int groupMinSessionTimeoutMs;
+
+    /**
+     * The group maximum session timeout.
+     */
+    private final int groupMaxSessionTimeoutMs;
+
+    /**
+     * The timer to add and cancel group operations.
      */
-    private TopicsImage topicsImage;
+    private final Timer<CompletableFuture<Errors>, Record> timer;
+
+    /**
+     * The time.
+     */
+    private final Time time;
 
     private GroupMetadataManager(
         SnapshotRegistry snapshotRegistry,
         LogContext logContext,
         List<PartitionAssignor> assignors,
-        TopicsImage topicsImage,
-        int consumerGroupMaxSize,
-        int consumerGroupHeartbeatIntervalMs
+        MetadataImage metadataImage,
+        TopicPartition topicPartition,
+        int groupMaxSize,
+        int consumerGroupHeartbeatIntervalMs,
+        int initialRebalanceDelayMs,
+        int newMemberJoinTimeoutMs,
+        int groupMinSessionTimeoutMs,
+        int groupMaxSessionTimeoutMs,
+        Timer<CompletableFuture<Errors>, Record> timer,
+        Time time
     ) {
+        this.logContext = logContext;
         this.log = logContext.logger(GroupMetadataManager.class);
         this.snapshotRegistry = snapshotRegistry;
-        this.topicsImage = topicsImage;
+        this.metadataImage = metadataImage;
         this.assignors = assignors.stream().collect(Collectors.toMap(PartitionAssignor::name, Function.identity()));
+        this.topicPartition = topicPartition;
         this.defaultAssignor = assignors.get(0);
         this.groups = new TimelineHashMap<>(snapshotRegistry, 0);
-        this.consumerGroupMaxSize = consumerGroupMaxSize;
+        this.groupMaxSize = groupMaxSize;
         this.consumerGroupHeartbeatIntervalMs = consumerGroupHeartbeatIntervalMs;
+        this.initialRebalanceDelayMs = initialRebalanceDelayMs;
+        this.newMemberJoinTimeoutMs = newMemberJoinTimeoutMs;
+        this.groupMinSessionTimeoutMs = groupMinSessionTimeoutMs;
+        this.groupMaxSessionTimeoutMs = groupMaxSessionTimeoutMs;
+        this.timer = timer;
+        this.time = time;
+    }
+
+    /**
+     * When a new metadata image is pushed.
+     *
+     * @param metadataImage The new metadata image.
+     */
+    public void onNewMetadataImage(MetadataImage metadataImage) {
+        this.metadataImage = metadataImage;
     }
 
     /**
      * Gets or maybe creates a consumer group.
      *
      * @param groupId           The group id.
+     * @param groupType         The group type (generic or consumer).
      * @param createIfNotExists A boolean indicating whether the group should be
      *                          created if it does not exist.
      *
      * @return A ConsumerGroup.
+     * @throws InvalidGroupIdException  if the group id is invalid.
      * @throws GroupIdNotFoundException if the group does not exist and createIfNotExists is false or
      *                                  if the group is not a consumer group.
      *
      * Package private for testing.
      */
-    ConsumerGroup getOrMaybeCreateConsumerGroup(
+    // Package private for testing.

Review Comment:
   nit: We can remove this one as there is the same phrase in the javadoc.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -171,70 +260,152 @@ GroupMetadataManager build() {
     /**
      * The maximum number of members allowed in a single consumer group.
      */
-    private final int consumerGroupMaxSize;
+    private final int groupMaxSize;
 
     /**
      * The heartbeat interval for consumer groups.
      */
     private final int consumerGroupHeartbeatIntervalMs;
 
     /**
-     * The topics metadata (or image).
+     * The metadata image.
+     */
+    private MetadataImage metadataImage;
+
+    // Rest of the fields are used for the generic group APIs.
+
+    /**
+     * An empty result returned to the state machine. This means that
+     * there are no records to append to the log.
+     *
+     * Package private for testing.
+     */
+    static final CoordinatorResult<CompletableFuture<Errors>, Record> EMPTY_RESULT =
+        new CoordinatorResult<>(Collections.emptyList(), CompletableFuture.completedFuture(null));
+
+    /**
+     * Initial rebalance delay for members joining a generic group.
+     */
+    private final int initialRebalanceDelayMs;
+
+    /**
+     * The timeout used to wait for a new member in milliseconds.
+     */
+    private final int newMemberJoinTimeoutMs;
+
+    /**
+     * The group minimum session timeout.
+     */
+    private final int groupMinSessionTimeoutMs;
+
+    /**
+     * The group maximum session timeout.
+     */
+    private final int groupMaxSessionTimeoutMs;
+
+    /**
+     * The timer to add and cancel group operations.
      */
-    private TopicsImage topicsImage;
+    private final Timer<CompletableFuture<Errors>, Record> timer;
+
+    /**
+     * The time.
+     */
+    private final Time time;
 
     private GroupMetadataManager(
         SnapshotRegistry snapshotRegistry,
         LogContext logContext,
         List<PartitionAssignor> assignors,
-        TopicsImage topicsImage,
-        int consumerGroupMaxSize,
-        int consumerGroupHeartbeatIntervalMs
+        MetadataImage metadataImage,
+        TopicPartition topicPartition,
+        int groupMaxSize,
+        int consumerGroupHeartbeatIntervalMs,
+        int initialRebalanceDelayMs,
+        int newMemberJoinTimeoutMs,
+        int groupMinSessionTimeoutMs,
+        int groupMaxSessionTimeoutMs,
+        Timer<CompletableFuture<Errors>, Record> timer,
+        Time time
     ) {
+        this.logContext = logContext;
         this.log = logContext.logger(GroupMetadataManager.class);
         this.snapshotRegistry = snapshotRegistry;
-        this.topicsImage = topicsImage;
+        this.metadataImage = metadataImage;
         this.assignors = assignors.stream().collect(Collectors.toMap(PartitionAssignor::name, Function.identity()));
+        this.topicPartition = topicPartition;
         this.defaultAssignor = assignors.get(0);
         this.groups = new TimelineHashMap<>(snapshotRegistry, 0);
-        this.consumerGroupMaxSize = consumerGroupMaxSize;
+        this.groupMaxSize = groupMaxSize;
         this.consumerGroupHeartbeatIntervalMs = consumerGroupHeartbeatIntervalMs;
+        this.initialRebalanceDelayMs = initialRebalanceDelayMs;
+        this.newMemberJoinTimeoutMs = newMemberJoinTimeoutMs;
+        this.groupMinSessionTimeoutMs = groupMinSessionTimeoutMs;
+        this.groupMaxSessionTimeoutMs = groupMaxSessionTimeoutMs;
+        this.timer = timer;
+        this.time = time;
+    }
+
+    /**
+     * When a new metadata image is pushed.
+     *
+     * @param metadataImage The new metadata image.
+     */
+    public void onNewMetadataImage(MetadataImage metadataImage) {
+        this.metadataImage = metadataImage;
     }
 
     /**
      * Gets or maybe creates a consumer group.
      *
      * @param groupId           The group id.
+     * @param groupType         The group type (generic or consumer).
      * @param createIfNotExists A boolean indicating whether the group should be
      *                          created if it does not exist.
      *
      * @return A ConsumerGroup.
+     * @throws InvalidGroupIdException  if the group id is invalid.
      * @throws GroupIdNotFoundException if the group does not exist and createIfNotExists is false or
      *                                  if the group is not a consumer group.
      *
      * Package private for testing.
      */
-    ConsumerGroup getOrMaybeCreateConsumerGroup(
+    // Package private for testing.
+    Group getOrMaybeCreateGroup(

Review Comment:
   I am not a fan of returning a `Group` here because it means that the caller have to cast the returned value. Is it possible to avoid it?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -171,70 +260,152 @@ GroupMetadataManager build() {
     /**
      * The maximum number of members allowed in a single consumer group.
      */
-    private final int consumerGroupMaxSize;
+    private final int groupMaxSize;
 
     /**
      * The heartbeat interval for consumer groups.
      */
     private final int consumerGroupHeartbeatIntervalMs;
 
     /**
-     * The topics metadata (or image).
+     * The metadata image.
+     */
+    private MetadataImage metadataImage;
+
+    // Rest of the fields are used for the generic group APIs.
+
+    /**
+     * An empty result returned to the state machine. This means that
+     * there are no records to append to the log.
+     *
+     * Package private for testing.
+     */
+    static final CoordinatorResult<CompletableFuture<Errors>, Record> EMPTY_RESULT =
+        new CoordinatorResult<>(Collections.emptyList(), CompletableFuture.completedFuture(null));
+
+    /**
+     * Initial rebalance delay for members joining a generic group.
+     */
+    private final int initialRebalanceDelayMs;
+
+    /**
+     * The timeout used to wait for a new member in milliseconds.
+     */
+    private final int newMemberJoinTimeoutMs;
+
+    /**
+     * The group minimum session timeout.
+     */
+    private final int groupMinSessionTimeoutMs;
+
+    /**
+     * The group maximum session timeout.
+     */
+    private final int groupMaxSessionTimeoutMs;
+
+    /**
+     * The timer to add and cancel group operations.
      */
-    private TopicsImage topicsImage;
+    private final Timer<CompletableFuture<Errors>, Record> timer;
+
+    /**
+     * The time.
+     */
+    private final Time time;
 
     private GroupMetadataManager(
         SnapshotRegistry snapshotRegistry,
         LogContext logContext,
         List<PartitionAssignor> assignors,
-        TopicsImage topicsImage,
-        int consumerGroupMaxSize,
-        int consumerGroupHeartbeatIntervalMs
+        MetadataImage metadataImage,
+        TopicPartition topicPartition,
+        int groupMaxSize,
+        int consumerGroupHeartbeatIntervalMs,
+        int initialRebalanceDelayMs,
+        int newMemberJoinTimeoutMs,
+        int groupMinSessionTimeoutMs,
+        int groupMaxSessionTimeoutMs,
+        Timer<CompletableFuture<Errors>, Record> timer,
+        Time time
     ) {
+        this.logContext = logContext;
         this.log = logContext.logger(GroupMetadataManager.class);
         this.snapshotRegistry = snapshotRegistry;
-        this.topicsImage = topicsImage;
+        this.metadataImage = metadataImage;
         this.assignors = assignors.stream().collect(Collectors.toMap(PartitionAssignor::name, Function.identity()));
+        this.topicPartition = topicPartition;
         this.defaultAssignor = assignors.get(0);
         this.groups = new TimelineHashMap<>(snapshotRegistry, 0);
-        this.consumerGroupMaxSize = consumerGroupMaxSize;
+        this.groupMaxSize = groupMaxSize;
         this.consumerGroupHeartbeatIntervalMs = consumerGroupHeartbeatIntervalMs;
+        this.initialRebalanceDelayMs = initialRebalanceDelayMs;
+        this.newMemberJoinTimeoutMs = newMemberJoinTimeoutMs;
+        this.groupMinSessionTimeoutMs = groupMinSessionTimeoutMs;
+        this.groupMaxSessionTimeoutMs = groupMaxSessionTimeoutMs;
+        this.timer = timer;
+        this.time = time;
+    }
+
+    /**
+     * When a new metadata image is pushed.
+     *
+     * @param metadataImage The new metadata image.
+     */
+    public void onNewMetadataImage(MetadataImage metadataImage) {
+        this.metadataImage = metadataImage;
     }
 
     /**
      * Gets or maybe creates a consumer group.
      *
      * @param groupId           The group id.
+     * @param groupType         The group type (generic or consumer).
      * @param createIfNotExists A boolean indicating whether the group should be
      *                          created if it does not exist.
      *
      * @return A ConsumerGroup.
+     * @throws InvalidGroupIdException  if the group id is invalid.
      * @throws GroupIdNotFoundException if the group does not exist and createIfNotExists is false or
      *                                  if the group is not a consumer group.
      *
      * Package private for testing.
      */
-    ConsumerGroup getOrMaybeCreateConsumerGroup(
+    // Package private for testing.
+    Group getOrMaybeCreateGroup(
         String groupId,
+        Group.GroupType groupType,
         boolean createIfNotExists
-    ) throws GroupIdNotFoundException {
+    ) throws InvalidGroupIdException, GroupIdNotFoundException {
+        if (groupId == null || groupId.isEmpty()) {
+            throw new InvalidGroupIdException(String.format("Group id %s is invalid.", groupId));
+        }

Review Comment:
   Should we move this to the request validation?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());

Review Comment:
   I suppose that being here means that group instance id is null.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {

Review Comment:
   nit: Would it make sense to update `plainProtocolSet` to take `request.protocols()`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {

Review Comment:
   What happens if the group instance id is an empty string?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata())).collect(Collectors.toList());

Review Comment:
   I have seen this code in a few place. It would be great to avoid it if possible.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata())).collect(Collectors.toList());
+
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(protocols))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(protocols)) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(protocols)) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            return tryCompleteJoinElseSchedule(group);
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Errors> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((error, t) -> {
+                    if (t != null || error != Errors.NONE) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        String message = t != null ? t.getMessage() : error.message();
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), message);
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member);
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {

Review Comment:
   Do we still need `hasSatisfiedHeartbeat` in the new model? If the timeout expires, it seems to me that it means that the member has failed to heartbeat in time; the timer would have been reset otherwise.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata())).collect(Collectors.toList());
+
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(protocols))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(protocols)) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(protocols)) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            return tryCompleteJoinElseSchedule(group);
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Errors> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((error, t) -> {
+                    if (t != null || error != Errors.NONE) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        String message = t != null ? t.getMessage() : error.message();
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), message);
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member);
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata()))
+            .collect(Collectors.toList());
+
+        group.updateMember(member, protocols, request.rebalanceTimeoutMs(), request.sessionTimeoutMs(), responseFuture);
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            protocols
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, newMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Errors>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = initialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - initialRebalanceDelayMs, 0);
+            group.setInitialRebalanceTimeoutMs(remainingMs);
+            group.setInitialRebalanceDelayMs(delayMs);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group)
+            );
+        }
+
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group
+    ) {
+        int remainingMs = group.initialRebalanceTimeoutMs();
+        int delayMs = group.initialRebalanceDelayMs();
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(initialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            group.setInitialRebalanceTimeoutMs(newRemainingMs);
+            group.setInitialRebalanceDelayMs(newDelayMs);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            group.setInitialRebalanceTimeoutMs(0);
+            group.setInitialRebalanceDelayMs(0);
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if ((member.assignment() == null || member.assignment().length == 0) && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat key from the timer and schedule
+     * a new heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        if (group.isInState(EMPTY) || group.isInState(DEAD)) {
+            // Always accept the request when the group is empty or dead
+            return true;
+
+        } else if (group.isInState(PREPARING_REBALANCE)) {
+            // An existing member is accepted if it is already awaiting. New members are accepted
+            // up to the max group size. Note that the number of awaiting members is used here
+            // for two reasons:
+            // 1) the group size is not reliable as it could already be above the max group size
+            //    if the max group size was reduced.
+            // 2) using the number of awaiting members allows to kick out the last rejoining
+            //    members of the group.
+            return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                group.numAwaitingJoinResponse() < groupMaxSize;
+
+        } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+            // An existing member is accepted. New members are accepted up to the max group size.
+            // Note that the group size is used here. When the group transitions to CompletingRebalance,
+            // members who haven't rejoined are removed.
+            return group.hasMemberId(memberId) || group.size() < groupMaxSize;
+        } else {
+            throw new IllegalStateException("Cannot handle state " + group.stateAsString());
+        }
+    }
+
+    /**
+     * Update a static member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group of the static member.
+     * @param oldMemberId     The existing static member id.
+     * @param newMemberId     The new joining static member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> updateStaticMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String oldMemberId,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String currentLeader = group.leaderOrNull();
+        GenericGroupMember member = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+
+        // Heartbeat of old member id will expire without effect since the group no longer contains that member id.
+        // New heartbeat shall be scheduled with new member id.
+        rescheduleGenericGroupMemberHeartbeat(group, member);
+
+        List<Protocol> protocols = new ArrayList<>(request.protocols().size());
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+
+        int oldRebalanceTimeoutMs = member.rebalanceTimeoutMs();
+        int oldSessionTimeoutMs = member.sessionTimeoutMs();
+        List<Protocol> oldProtocols = member.supportedProtocols();
+
+        group.updateMember(
+            member,
+            protocols,
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        if (group.isInState(STABLE)) {
+            // Check if group's selected protocol of next generation will change, if not, simply store group to persist
+            // the updated static member, if yes, rebalance should be triggered to keep the group's assignment
+            // and selected protocol consistent
+            String groupInstanceId = request.groupInstanceId();
+            String selectedProtocolForNextGeneration = group.selectProtocol();
+            if (group.protocolName().orElse("").equals(selectedProtocolForNextGeneration)) {
+                log.info("Static member which joins during Stable stage and doesn't affect " +
+                    "the selected protocol will not trigger a rebalance.");
+
+                CompletableFuture<Errors> appendFuture = new CompletableFuture<>();

Review Comment:
   Shouldn't we use `CompletableFuture<Void>` and complete the future exceptionally with the exception corresponding to the error?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata())).collect(Collectors.toList());
+
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(protocols))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(protocols)) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(protocols)) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            return tryCompleteJoinElseSchedule(group);
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Errors> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((error, t) -> {
+                    if (t != null || error != Errors.NONE) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        String message = t != null ? t.getMessage() : error.message();
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), message);
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member);
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {

Review Comment:
   i think we can cancel the existing heartbeat when a member rejoins during a rebalance. this will still expire if the member does not rejoin which is what we want. also, moving the new member join timeout to a different key will help remove this `hasSatisfiedHeartbeat`.
   
   is that what you had in mind?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }

Review Comment:
   This is now further worsened with the new records on creating a group.
   
   The only "non-hacky" approach i can think of is just returning 
   `List<CoordinatorResult<CompletableFuture<Void>, Record>`
   
   which the runtime would append & commit then complete in order. but this adds a lot of complexity for something we actually won't use in practice.
   
   The other approach (which i have implemented) is to ignore the result from `completeGenericGroupJoin` when invoked from the join group path.
   
   This works because `completeGenericGroupJoin` only produces records when a member expires.
   Also, when a new group is created we don't have any other records to append.
   
   However, this still feels a bit hacky. Not sure how to resolve 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -171,70 +260,152 @@ GroupMetadataManager build() {
     /**
      * The maximum number of members allowed in a single consumer group.
      */
-    private final int consumerGroupMaxSize;
+    private final int groupMaxSize;
 
     /**
      * The heartbeat interval for consumer groups.
      */
     private final int consumerGroupHeartbeatIntervalMs;
 
     /**
-     * The topics metadata (or image).
+     * The metadata image.
+     */
+    private MetadataImage metadataImage;
+
+    // Rest of the fields are used for the generic group APIs.
+
+    /**
+     * An empty result returned to the state machine. This means that
+     * there are no records to append to the log.
+     *
+     * Package private for testing.
+     */
+    static final CoordinatorResult<CompletableFuture<Errors>, Record> EMPTY_RESULT =
+        new CoordinatorResult<>(Collections.emptyList(), CompletableFuture.completedFuture(null));
+
+    /**
+     * Initial rebalance delay for members joining a generic group.
+     */
+    private final int initialRebalanceDelayMs;
+
+    /**
+     * The timeout used to wait for a new member in milliseconds.
+     */
+    private final int newMemberJoinTimeoutMs;
+
+    /**
+     * The group minimum session timeout.
+     */
+    private final int groupMinSessionTimeoutMs;
+
+    /**
+     * The group maximum session timeout.
+     */
+    private final int groupMaxSessionTimeoutMs;
+
+    /**
+     * The timer to add and cancel group operations.
      */
-    private TopicsImage topicsImage;
+    private final Timer<CompletableFuture<Errors>, Record> timer;
+
+    /**
+     * The time.
+     */
+    private final Time time;
 
     private GroupMetadataManager(
         SnapshotRegistry snapshotRegistry,
         LogContext logContext,
         List<PartitionAssignor> assignors,
-        TopicsImage topicsImage,
-        int consumerGroupMaxSize,
-        int consumerGroupHeartbeatIntervalMs
+        MetadataImage metadataImage,
+        TopicPartition topicPartition,
+        int groupMaxSize,
+        int consumerGroupHeartbeatIntervalMs,
+        int initialRebalanceDelayMs,
+        int newMemberJoinTimeoutMs,
+        int groupMinSessionTimeoutMs,
+        int groupMaxSessionTimeoutMs,
+        Timer<CompletableFuture<Errors>, Record> timer,
+        Time time
     ) {
+        this.logContext = logContext;
         this.log = logContext.logger(GroupMetadataManager.class);
         this.snapshotRegistry = snapshotRegistry;
-        this.topicsImage = topicsImage;
+        this.metadataImage = metadataImage;
         this.assignors = assignors.stream().collect(Collectors.toMap(PartitionAssignor::name, Function.identity()));
+        this.topicPartition = topicPartition;
         this.defaultAssignor = assignors.get(0);
         this.groups = new TimelineHashMap<>(snapshotRegistry, 0);
-        this.consumerGroupMaxSize = consumerGroupMaxSize;
+        this.groupMaxSize = groupMaxSize;
         this.consumerGroupHeartbeatIntervalMs = consumerGroupHeartbeatIntervalMs;
+        this.initialRebalanceDelayMs = initialRebalanceDelayMs;
+        this.newMemberJoinTimeoutMs = newMemberJoinTimeoutMs;
+        this.groupMinSessionTimeoutMs = groupMinSessionTimeoutMs;
+        this.groupMaxSessionTimeoutMs = groupMaxSessionTimeoutMs;
+        this.timer = timer;
+        this.time = time;
+    }
+
+    /**
+     * When a new metadata image is pushed.
+     *
+     * @param metadataImage The new metadata image.
+     */
+    public void onNewMetadataImage(MetadataImage metadataImage) {
+        this.metadataImage = metadataImage;
     }
 
     /**
      * Gets or maybe creates a consumer group.
      *
      * @param groupId           The group id.
+     * @param groupType         The group type (generic or consumer).
      * @param createIfNotExists A boolean indicating whether the group should be
      *                          created if it does not exist.
      *
      * @return A ConsumerGroup.
+     * @throws InvalidGroupIdException  if the group id is invalid.
      * @throws GroupIdNotFoundException if the group does not exist and createIfNotExists is false or
      *                                  if the group is not a consumer group.
      *
      * Package private for testing.
      */
-    ConsumerGroup getOrMaybeCreateConsumerGroup(
+    // Package private for testing.
+    Group getOrMaybeCreateGroup(

Review Comment:
   i think we should store generic groups separately. the added benefit here is that we wouldn't have to create a new record when a new group was created as you have mentioned in the comment below. wdyt?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +295,21 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isValidGroupId(request.groupId(), ApiKeys.forId(request.apiKey()))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleWriteOperation("generic-group-join",

Review Comment:
   yeah, i will log an error for 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -376,19 +448,28 @@ public CoordinatorResult<ConsumerGroupHeartbeatResponseData, Record> consumerGro
             return result;
         }
 
-        public List<MockCoordinatorTimer.ExpiredTimeout<Record>> sleep(long ms) {
+        public List<ExpiredTimeout<Void, Record>> sleep(long ms) {
             time.sleep(ms);
-            List<MockCoordinatorTimer.ExpiredTimeout<Record>> timeouts = timer.poll();
-            timeouts.forEach(timeout -> timeout.records.forEach(this::replay));
+            List<ExpiredTimeout<Void, Record>> timeouts = timer.poll();
+            timeouts.forEach(timeout -> {
+                if (timeout.result.replayRecords()) {
+                    timeout.result.records().forEach(this::replay);
+                }
+            });
             return timeouts;
         }
 
-        public MockCoordinatorTimer.ScheduledTimeout<Record> assertSessionTimeout(
+        public void sleepAndAssertEmptyResult(long ms) {
+            List<ExpiredTimeout<Void, Record>> timeouts = sleep(ms);
+            timeouts.forEach(timeout -> assertEquals(EMPTY_RESULT, timeout.result));
+        }

Review Comment:
   will keep it as assertEmptyResult as the timeout is not empty (can be) but we want to assert that the coordinator result is.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -3022,143 +3242,2087 @@ public void testOnLoaded() {
         assertNotNull(context.timer.timeout(consumerGroupRevocationTimeoutKey("foo", "foo-1")));
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
-
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
-        }
-    }
+    @Test
+    public void testGenerateRecordsOnNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get().errorCode());
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        GenericGroup group = context.createGenericGroup("group-id");
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        assertEquals(
+            Collections.singletonList(RecordHelpers.newEmptyGroupMetadataRecord(group, MetadataVersion.latest())),
+            result.records()
+        );
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
+        context.createGenericGroup("group-id");
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        IntStream.range(0, 10).forEach(i -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+            assertFalse(responseFuture.isDone());
+            assertTrue(result.records().isEmpty());
         });
-        return assignmentMap;
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        GenericGroup group = context.createGenericGroup("group-id");
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> firstRoundFutures = new ArrayList<>();
+        IntStream.range(0, groupMaxSize + 1).forEach(i -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            firstRoundFutures.add(responseFuture);
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture, requiredKnownMemberId);
+            assertTrue(responseFuture.isDone());
+            try {
+                assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get().errorCode());
+            } catch (Exception ignored) {

Review Comment:
   Do we really need to keep the try..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] dajac commented on a diff in pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -5235,6 +5316,11 @@ private List<String> verifyGenericGroupJoinResponses(
         return memberIds;
     }
 
+    private void assertEmptyResult(List<ExpiredTimeout<Void, Record>> timeouts) {

Review Comment:
   nit: `assertNoOrEmptyResult`?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -561,7 +625,7 @@ public boolean hasReceivedSyncFromAllMembers() {
      * @return members that have yet to sync.
      */
     public Set<String> allPendingSyncMembers() {
-        return pendingSyncMembers;
+        return new HashSet<>(pendingSyncMembers);

Review Comment:
   in `GroupMetadataManager#expirePendingSync()` we remove members from the set while iterating



-- 
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 pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

Posted by "dajac (via GitHub)" <gi...@apache.org>.
dajac commented on PR #13870:
URL: https://github.com/apache/kafka/pull/13870#issuecomment-1632143234

   @jeffkbkim Now that https://github.com/apache/kafka/pull/13963 is merged, could you update your PR?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1245,4 +1422,1304 @@ public static String consumerGroupSessionTimeoutKey(String groupId, String membe
     public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) {
         return "revocation-timeout-" + groupId + "-" + memberId;
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            removeGroup(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> genericGroup.add(member, null));
+            groups.put(groupId, genericGroup);
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if a group was newly created, we need to append
+                // records to the log to commit the group to the timeline data structure.
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setErrorCode(appendGroupMetadataErrorToResponseError(Errors.forException(t)).code()));
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newEmptyGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Attempt to complete join group phase. We do not complete
+     * the join group phase if this is the initial rebalance.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinPhase(GenericGroup group) {
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                responseFuture
+            );
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (JoinGroupRequest.requiresKnownMemberId(context.apiVersion())) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String genericGroupHeartbeatKey = genericGroupHeartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                genericGroupHeartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberThenRebalanceOrCompleteJoin(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " + group.stateAsString() +
+                            "; client reason: " + JoinGroupRequest.joinReason(request),
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(group.isLeader(memberId) ?
+                                group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(genericGroupJoinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(genericGroupHeartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                genericGroupJoinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}) with {} members",
+                    groupId, group.generationId(), topicPartition, group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(genericGroupHeartbeatKey(groupId, member.memberId()));

Review Comment:
   @jeffkbkim That's 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -367,6 +446,155 @@ public CoordinatorResult<ConsumerGroupHeartbeatResponseData, Record> consumerGro
             return result;
         }
 
+        public CompletableFuture<JoinGroupResponseData> sendGenericGroupJoin(
+            JoinGroupRequestData request
+        ) {
+            return sendGenericGroupJoin(request, false);
+        }
+
+        public CompletableFuture<JoinGroupResponseData> sendGenericGroupJoin(
+            JoinGroupRequestData request,
+            boolean requireKnownMemberId
+        ) {
+            return sendGenericGroupJoin(request, requireKnownMemberId, false, null);
+        }
+
+        public CompletableFuture<JoinGroupResponseData> sendGenericGroupJoin(
+            JoinGroupRequestData request,
+            boolean requireKnownMemberId,
+            boolean supportSkippingAssignment,
+            ExpectedGenericGroupResult expectedResult
+        ) {
+            // requireKnownMemberId is true: version >= 4
+            // supportSkippingAssignment is true: version >= 9
+            short joinGroupVersion = 3;
+
+            if (requireKnownMemberId) {
+                joinGroupVersion = 4;
+                if (supportSkippingAssignment) {
+                    joinGroupVersion = ApiKeys.JOIN_GROUP.latestVersion();
+                }
+            }
+
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+            RequestContext context = new RequestContext(
+                new RequestHeader(
+                    ApiKeys.JOIN_GROUP,
+                    joinGroupVersion,
+                    "client",
+                    0
+                ),
+                "1",
+                InetAddress.getLoopbackAddress(),
+                KafkaPrincipal.ANONYMOUS,
+                ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
+                SecurityProtocol.PLAINTEXT,
+                ClientInformation.EMPTY,
+                false
+            );
+
+            CoordinatorResult<Void, Record> result = groupMetadataManager.genericGroupJoin(
+                context,
+                request,
+                responseFuture
+            );
+
+            if (expectedResult != null) {
+                GenericGroup group = groupMetadataManager.getOrMaybeCreateGenericGroup(
+                    request.groupId(),
+                    false
+                );
+
+                Record groupMetadataRecord;
+                if (expectedResult.isNewGroup) {
+                    groupMetadataRecord = newEmptyGroupMetadataRecord(group, MetadataVersion.latest());
+                } else {
+                    groupMetadataRecord = RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest());
+                }
+
+                expectedResult.records = Collections.singletonList(groupMetadataRecord);

Review Comment:
   i'll think a bit more on this as it will require a large change in this class.
   
   one of the reasons i had it like this is that we mostly care about the responseFuture in the tests and wanted to hide the record/append future validations. The timer could also produce records which require setting things in advance.
   
   i agree it is unclean, i'll address this in the next commit.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata())).collect(Collectors.toList());
+
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(protocols))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(protocols)) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(protocols)) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            return tryCompleteJoinElseSchedule(group);
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Errors> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((error, t) -> {
+                    if (t != null || error != Errors.NONE) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        String message = t != null ? t.getMessage() : error.message();
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), message);
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member);
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata()))
+            .collect(Collectors.toList());
+
+        group.updateMember(member, protocols, request.rebalanceTimeoutMs(), request.sessionTimeoutMs(), responseFuture);
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            protocols
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, newMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Errors>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = initialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - initialRebalanceDelayMs, 0);
+            group.setInitialRebalanceTimeoutMs(remainingMs);
+            group.setInitialRebalanceDelayMs(delayMs);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group)
+            );
+        }
+
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group
+    ) {
+        int remainingMs = group.initialRebalanceTimeoutMs();
+        int delayMs = group.initialRebalanceDelayMs();
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(initialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            group.setInitialRebalanceTimeoutMs(newRemainingMs);
+            group.setInitialRebalanceDelayMs(newDelayMs);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group)

Review Comment:
   initial rebalances don't have check and complete as try complete always returns false so this works. thanks



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +285,31 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isGroupIdNotEmpty(request.groupId())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleWriteOperation("generic-group-join",
+            topicPartitionFor(request.groupId()),
+            coordinator -> coordinator.genericGroupJoin(context, request, responseFuture)
+        ).exceptionally(exception -> {
+            log.error("Request {} hit an unexpected exception: {}. ",
+                request, exception.getMessage());
+
+            if (exception instanceof IllegalStateException) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()));
+            }

Review Comment:
   updated to all errors. 



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );

Review Comment:
   this is not to create a new group (note the `createIfNotExists=false` argument) but to retrieve the group to do more validations such as group state, generation id, group size, etc.
   
   added a helper method `genericGroup()` to simplify the calls.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }
-    }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, new ArrayList<>(group.allMembers()).get(0).memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());
+        assertNull(response.protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertFalse(responseFuture.isDone());
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+        context.timer.advanceClock(2);
+
+        // Advance clock past initial rebalance delay and verify futures are not completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+
+        // Advance clock beyond recomputed delay and make sure the futures have completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldDelayRebalanceUptoRebalanceTimeout() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs + 1);
+
+        CompletableFuture<JoinGroupResponseData> thirdMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        // Advance clock right before rebalance timeout.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+        assertFalse(thirdMemberResponseFuture.isDone());
+
+        // Advance clock beyond rebalance timeout.
+        context.timer.advanceClock(1);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertTrue(thirdMemberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupReplaceStaticMember() throws Exception {

Review Comment:
   yes, that's 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1230,1358 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = groups.get(groupId) == null;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We create records here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()));
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                newGroupResult = new CoordinatorResult<>(records, appendFuture);
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if the group is new, we need to
+                // persist the group to the timeline map.
+                result = newGroupResult;
+            }
+        }
+        return result;
+    }
+
+    private CoordinatorResult<Void, Record> tryCompleteJoinPhase(GenericGroup group) {
+        // Attempt to complete join group phase. We do not complete
+        // the join group phase if this is the initial rebalance.
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(heartbeatKey(groupId, member.memberId()));
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalanceOrCompleteJoin(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalanceOrCompleteJoin(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.previousState() == EMPTY) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalanceOrCompleteJoin(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state. Otherwise, try
+     * to complete the join phase.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybePrepareRebalanceOrCompleteJoin(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        } else {
+            return tryCompleteJoinPhase(group);
+        }
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<Void, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.isInState(EMPTY);
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if (!member.hasAssignment() && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<Void, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        switch (group.currentState()) {
+            case EMPTY:
+            case DEAD:
+                // Always accept the request when the group is empty or dead
+                return true;
+            case PREPARING_REBALANCE:
+                // An existing member is accepted if it is already awaiting. New members are accepted
+                // up to the max group size. Note that the number of awaiting members is used here
+                // for two reasons:
+                // 1) the group size is not reliable as it could already be above the max group size
+                //    if the max group size was reduced.
+                // 2) using the number of awaiting members allows to kick out the last rejoining
+                //    members of the group.
+                return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                    group.numAwaitingJoinResponse() < genericGroupMaxSize;
+            case COMPLETING_REBALANCE:
+            case STABLE:
+                // An existing member is accepted. New members are accepted up to the max group size.
+                // Note that the group size is used here. When the group transitions to CompletingRebalance,
+                // members who haven't rejoined are removed.
+                return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+            default:
+                throw new IllegalStateException("Unknown group state: " + group.stateAsString());

Review Comment:
   i added this in case we add a new state in the future. should i remove 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1243,4 +1406,1283 @@ public static String consumerGroupSessionTimeoutKey(String groupId, String membe
     public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) {
         return "revocation-timeout-" + groupId + "-" + memberId;
     }
+
+     /** Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if a group was newly created, we need to append
+                // records to the log to commit the group to the timeline data structure.
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newEmptyGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Attempt to complete join group phase. We do not complete
+     * the join group phase if this is the initial rebalance.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinPhase(GenericGroup group) {
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                responseFuture
+            );
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (JoinGroupRequest.requiresKnownMemberId(context.apiVersion())) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberThenRebalanceOrCompleteJoin(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " + group.stateAsString() +
+                            "; client reason: " + JoinGroupRequest.joinReason(request),
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(group.isLeader(memberId) ?
+                                group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}) with {} members",
+                    groupId, group.generationId(), topicPartition, group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(heartbeatKey(groupId, member.memberId()));
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            false,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalanceOrCompleteJoin(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member. Then begin a rebalance or complete the join phase.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateMemberThenRebalanceOrCompleteJoin(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalanceOrCompleteJoin(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.previousState() == EMPTY) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalanceOrCompleteJoin(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + JoinGroupRequest.joinReason(request));
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state. Otherwise, try
+     * to complete the join phase.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybePrepareRebalanceOrCompleteJoin(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        } else {
+            return maybeCompleteJoinPhase(group);
+        }
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     *
+     * Package private for testing.
+     */
+    CoordinatorResult<Void, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.isInState(EMPTY);
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : maybeCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if (!member.hasAssignment() && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            false,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<Void, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {

Review Comment:
   to confirm, you're saying we should call `acceptJoiningMember` while loading 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1245,4 +1422,1304 @@ public static String consumerGroupSessionTimeoutKey(String groupId, String membe
     public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) {
         return "revocation-timeout-" + groupId + "-" + memberId;
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            removeGroup(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> genericGroup.add(member, null));
+            groups.put(groupId, genericGroup);
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if a group was newly created, we need to append
+                // records to the log to commit the group to the timeline data structure.
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setErrorCode(appendGroupMetadataErrorToResponseError(Errors.forException(t)).code()));
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newEmptyGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Attempt to complete join group phase. We do not complete
+     * the join group phase if this is the initial rebalance.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinPhase(GenericGroup group) {
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                responseFuture
+            );
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (JoinGroupRequest.requiresKnownMemberId(context.apiVersion())) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String genericGroupHeartbeatKey = genericGroupHeartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                genericGroupHeartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberThenRebalanceOrCompleteJoin(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " + group.stateAsString() +
+                            "; client reason: " + JoinGroupRequest.joinReason(request),
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(group.isLeader(memberId) ?
+                                group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(genericGroupJoinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(genericGroupHeartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                genericGroupJoinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}) with {} members",
+                    groupId, group.generationId(), topicPartition, group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(genericGroupHeartbeatKey(groupId, member.memberId()));

Review Comment:
   So we actually need to reschedule the timer here, right?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1243,4 +1406,1283 @@ public static String consumerGroupSessionTimeoutKey(String groupId, String membe
     public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) {
         return "revocation-timeout-" + groupId + "-" + memberId;
     }
+
+     /** Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if a group was newly created, we need to append
+                // records to the log to commit the group to the timeline data structure.
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());

Review Comment:
   yeah, possibly.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +282,32 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isGroupIdNotEmpty(request.groupId())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleWriteOperation("generic-group-join",
+            topicPartitionFor(request.groupId()),
+            coordinator -> coordinator.genericGroupJoin(context, request, responseFuture)
+        ).exceptionally(exception -> {
+            log.error("Request {} hit an unexpected exception: {}",
+                request, exception.getMessage());

Review Comment:
   This was not addressed.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata())).collect(Collectors.toList());
+
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(protocols))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(protocols)) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(protocols)) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            return tryCompleteJoinElseSchedule(group);
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Errors> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((error, t) -> {
+                    if (t != null || error != Errors.NONE) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        String message = t != null ? t.getMessage() : error.message();
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), message);
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member);

Review Comment:
   are you referring to any lingering heartbeats at this point? i think we can just cancel them here right



-- 
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 pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on PR #13870:
URL: https://github.com/apache/kafka/pull/13870#issuecomment-1633127990

   @dajac I have updated with latest and unified the MockCoordinatorTimer.


-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -247,31 +324,78 @@ GroupMetadataManager build() {
      */
     private MetadataImage metadataImage;
 
+    /**
+     * An empty result returned to the state machine. This means that
+     * there are no records to append to the log.
+     *
+     * Package private for testing.
+     */
+    static final CoordinatorResult<Void, Record> EMPTY_RESULT =
+        new CoordinatorResult<>(Collections.emptyList(), CompletableFuture.completedFuture(null));

Review Comment:
   ah.. did not think about that.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +295,21 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isValidGroupId(request.groupId(), ApiKeys.forId(request.apiKey()))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleGenericGroupOperation("generic-group-join",
+            topicPartitionFor(request.groupId()),
+            coordinator -> coordinator.genericGroupJoin(context, request, responseFuture));
+
+        return responseFuture;

Review Comment:
   i'm a bit confused, is the coordinator write operation future the result we wait for committing?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -171,70 +260,152 @@ GroupMetadataManager build() {
     /**
      * The maximum number of members allowed in a single consumer group.
      */
-    private final int consumerGroupMaxSize;
+    private final int groupMaxSize;
 
     /**
      * The heartbeat interval for consumer groups.
      */
     private final int consumerGroupHeartbeatIntervalMs;
 
     /**
-     * The topics metadata (or image).
+     * The metadata image.
+     */
+    private MetadataImage metadataImage;
+
+    // Rest of the fields are used for the generic group APIs.
+
+    /**
+     * An empty result returned to the state machine. This means that
+     * there are no records to append to the log.
+     *
+     * Package private for testing.
+     */
+    static final CoordinatorResult<CompletableFuture<Errors>, Record> EMPTY_RESULT =
+        new CoordinatorResult<>(Collections.emptyList(), CompletableFuture.completedFuture(null));
+
+    /**
+     * Initial rebalance delay for members joining a generic group.
+     */
+    private final int initialRebalanceDelayMs;
+
+    /**
+     * The timeout used to wait for a new member in milliseconds.
+     */
+    private final int newMemberJoinTimeoutMs;
+
+    /**
+     * The group minimum session timeout.
+     */
+    private final int groupMinSessionTimeoutMs;
+
+    /**
+     * The group maximum session timeout.
+     */
+    private final int groupMaxSessionTimeoutMs;
+
+    /**
+     * The timer to add and cancel group operations.
      */
-    private TopicsImage topicsImage;
+    private final Timer<CompletableFuture<Errors>, Record> timer;
+
+    /**
+     * The time.
+     */
+    private final Time time;
 
     private GroupMetadataManager(
         SnapshotRegistry snapshotRegistry,
         LogContext logContext,
         List<PartitionAssignor> assignors,
-        TopicsImage topicsImage,
-        int consumerGroupMaxSize,
-        int consumerGroupHeartbeatIntervalMs
+        MetadataImage metadataImage,
+        TopicPartition topicPartition,
+        int groupMaxSize,
+        int consumerGroupHeartbeatIntervalMs,
+        int initialRebalanceDelayMs,
+        int newMemberJoinTimeoutMs,
+        int groupMinSessionTimeoutMs,
+        int groupMaxSessionTimeoutMs,
+        Timer<CompletableFuture<Errors>, Record> timer,
+        Time time
     ) {
+        this.logContext = logContext;
         this.log = logContext.logger(GroupMetadataManager.class);
         this.snapshotRegistry = snapshotRegistry;
-        this.topicsImage = topicsImage;
+        this.metadataImage = metadataImage;
         this.assignors = assignors.stream().collect(Collectors.toMap(PartitionAssignor::name, Function.identity()));
+        this.topicPartition = topicPartition;
         this.defaultAssignor = assignors.get(0);
         this.groups = new TimelineHashMap<>(snapshotRegistry, 0);
-        this.consumerGroupMaxSize = consumerGroupMaxSize;
+        this.groupMaxSize = groupMaxSize;
         this.consumerGroupHeartbeatIntervalMs = consumerGroupHeartbeatIntervalMs;
+        this.initialRebalanceDelayMs = initialRebalanceDelayMs;
+        this.newMemberJoinTimeoutMs = newMemberJoinTimeoutMs;
+        this.groupMinSessionTimeoutMs = groupMinSessionTimeoutMs;
+        this.groupMaxSessionTimeoutMs = groupMaxSessionTimeoutMs;
+        this.timer = timer;
+        this.time = time;
+    }
+
+    /**
+     * When a new metadata image is pushed.
+     *
+     * @param metadataImage The new metadata image.
+     */
+    public void onNewMetadataImage(MetadataImage metadataImage) {
+        this.metadataImage = metadataImage;
     }
 
     /**
      * Gets or maybe creates a consumer group.
      *
      * @param groupId           The group id.
+     * @param groupType         The group type (generic or consumer).
      * @param createIfNotExists A boolean indicating whether the group should be
      *                          created if it does not exist.
      *
      * @return A ConsumerGroup.
+     * @throws InvalidGroupIdException  if the group id is invalid.
      * @throws GroupIdNotFoundException if the group does not exist and createIfNotExists is false or
      *                                  if the group is not a consumer group.
      *
      * Package private for testing.
      */
-    ConsumerGroup getOrMaybeCreateConsumerGroup(
+    // Package private for testing.
+    Group getOrMaybeCreateGroup(
         String groupId,
+        Group.GroupType groupType,
         boolean createIfNotExists
-    ) throws GroupIdNotFoundException {
+    ) throws InvalidGroupIdException, GroupIdNotFoundException {
+        if (groupId == null || groupId.isEmpty()) {
+            throw new InvalidGroupIdException(String.format("Group id %s is invalid.", groupId));
+        }

Review Comment:
   i think that static validation could be done in the group coordinator service; however we have to keep the validation which depends on internal values in the state machine.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;

Review Comment:
   see https://github.com/apache/kafka/pull/13870#discussion_r1235653740



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());

Review Comment:
   you're right, removed. 



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1348 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (group.isNew()) {

Review Comment:
   will revisit this after refactoring the CoordinatorResult return type / generate multiple records discussion.
   
   the reason that a field was set is because where we add the group to `groups` (and initialize the append future) and where we set it as a return type (L1265) are at different places. once the group is added we can't check via groups.get(groupId).



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1230,1358 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = groups.get(groupId) == null;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log

Review Comment:
   the issue is that the records need to be generated while the group is empty. after performing `genericGroupJoinNewMember()` the group will have added the member metadata. the existing protocol only allows records for empty groups or groups that have a defined protocol.
   
   this only applies to join group requests with `requireKnownMemberId = false` or 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] jeffkbkim commented on a diff in pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1230,1358 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = groups.get(groupId) == null;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We create records here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()));

Review Comment:
   the main concern i had was completeGenericGroupJoin() can be invoked by the timer which would miss this conversion but i guess it's not really an issue. 



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1221,1331 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We create records here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());

Review Comment:
   that's 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] dajac commented on a diff in pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +295,21 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isValidGroupId(request.groupId(), ApiKeys.forId(request.apiKey()))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleWriteOperation("generic-group-join",

Review Comment:
   I wonder if we need to handle the future returned by `scheduleWriteOperation` as well. At minimum, we may want to react to errors. This could for instance happen if something goes wrong before the join group handling is event triggered.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -578,4 +619,19 @@ public void shutdown() {
         Utils.closeQuietly(runtime, "coordinator runtime");
         log.info("Shutdown complete.");
     }
+
+    private boolean isValidGroupId(String groupId, ApiKeys api) {
+        if (api == ApiKeys.OFFSET_COMMIT ||
+            api == ApiKeys.OFFSET_FETCH ||
+            api == ApiKeys.DESCRIBE_GROUPS ||
+            api == ApiKeys.DELETE_GROUPS

Review Comment:
   nit: I am not a fan of this validation. I wonder if we should just have two helpers: `isGroupIdNotNull` and `isGroupIdNotEmpty`. In this PR, we would only need `isGroupIdNotEmpty`. What do you think?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1348 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();

Review Comment:
   I wonder if we could avoid passing the version to this method by adding `-1` as the default value of `rebalanceTimeout` in `GroupMetadataValue`. It seems that we could rely on this to decide here.
   
   Another way that I was thinking about would be to pass the `Record` to the replay method as it contains all the available information. Have you considered this?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1348 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.

Review Comment:
   I think that the group should be deleted in this case.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +295,21 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isValidGroupId(request.groupId(), ApiKeys.forId(request.apiKey()))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleGenericGroupOperation("generic-group-join",
+            topicPartitionFor(request.groupId()),
+            coordinator -> coordinator.genericGroupJoin(context, request, responseFuture));
+
+        return responseFuture;

Review Comment:
   Sorry, I was not clear. I meant that we need to port this [logic](https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/coordinator/group/GroupMetadataManager.scala#L288) here.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1348 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (group.isNew()) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We set this here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                newGroupResult = new CoordinatorResult<>(records, appendFuture);
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                CoordinatorResult<Void, Record> joinResult = completeGenericGroupJoin(group);
+                result = result != EMPTY_RESULT ? result : joinResult;
+            }
+
+            if (group.isNew() && result == EMPTY_RESULT) {
+                // If there are no records to append, we need to append
+                // records for the new group.
+                result = newGroupResult;
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(heartbeatKey(groupId, member.memberId()));
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<Void, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if (!member.hasAssignment() && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<Void, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        switch (group.currentState()) {
+            case EMPTY:
+            case DEAD:
+                // Always accept the request when the group is empty or dead
+                return true;
+            case PREPARING_REBALANCE:
+                // An existing member is accepted if it is already awaiting. New members are accepted
+                // up to the max group size. Note that the number of awaiting members is used here
+                // for two reasons:
+                // 1) the group size is not reliable as it could already be above the max group size
+                //    if the max group size was reduced.
+                // 2) using the number of awaiting members allows to kick out the last rejoining
+                //    members of the group.
+                return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                    group.numAwaitingJoinResponse() < genericGroupMaxSize;
+            case COMPLETING_REBALANCE:
+            case STABLE:
+                // An existing member is accepted. New members are accepted up to the max group size.
+                // Note that the group size is used here. When the group transitions to CompletingRebalance,
+                // members who haven't rejoined are removed.
+                return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+            default:
+                throw new IllegalStateException("Unknown group state: " + group.stateAsString());
+        }
+    }
+
+    /**
+     * Update a static member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group of the static member.
+     * @param oldMemberId     The existing static member id.
+     * @param newMemberId     The new joining static member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateStaticMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String oldMemberId,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String currentLeader = group.leaderOrNull();
+        GenericGroupMember member = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+
+        // Heartbeat of old member id will expire without effect since the group no longer contains that member id.
+        // New heartbeat shall be scheduled with new member id.
+        rescheduleGenericGroupMemberHeartbeat(group, member);
+
+        int oldRebalanceTimeoutMs = member.rebalanceTimeoutMs();
+        int oldSessionTimeoutMs = member.sessionTimeoutMs();
+        JoinGroupRequestProtocolCollection oldProtocols = member.supportedProtocols();
+
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        if (group.isInState(STABLE)) {
+            // Check if group's selected protocol of next generation will change, if not, simply store group to persist
+            // the updated static member, if yes, rebalance should be triggered to keep the group's assignment
+            // and selected protocol consistent
+            String groupInstanceId = request.groupInstanceId();
+            String selectedProtocolForNextGeneration = group.selectProtocol();
+            if (group.protocolName().orElse("").equals(selectedProtocolForNextGeneration)) {
+                log.info("Static member which joins during Stable stage and doesn't affect " +
+                    "the selected protocol will not trigger a rebalance.");
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    // Starting from version 9 of the JoinGroup API, static members are able to
+                    // skip running the assignor based on the `SkipAssignment` field. We leverage
+                    // this to tell the leader that it is the leader of the group but by skipping
+                    // running the assignor while the group is in stable state.
+                    // Notes:
+                    // 1) This allows the leader to continue monitoring metadata changes for the
+                    // group. Note that any metadata changes happening while the static leader is
+                    // down won't be noticed.
+                    // 2) The assignors are not idempotent nor free from side effects. This is why
+                    // we skip entirely the assignment step as it could generate a different group
+                    // assignment which would be ignored by the group coordinator because the group
+                    // is the stable state.
+                    boolean supportSkippingAssignment = context.apiVersion() >= 9;
+
+                    if (t != null) {
+                        log.warn("Failed to persist metadata for group {}: {}", group.groupId(), t.getMessage());
+
+                        // Failed to persist the member id of the given static member, revert the update of the static member in the group.
+                        group.updateMember(member, oldProtocols, oldRebalanceTimeoutMs, oldSessionTimeoutMs, null);
+                        GenericGroupMember oldMember = group.replaceStaticMember(groupInstanceId, newMemberId, oldMemberId);
+                        rescheduleGenericGroupMemberHeartbeat(group, oldMember);
+
+                        short errorCode = Errors.forException(t).code();
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setMembers(Collections.emptyList())
+                                .setMemberId(UNKNOWN_MEMBER_ID)
+                                .setGenerationId(group.generationId())
+                                .setProtocolName(group.protocolName().orElse(null))
+                                .setProtocolType(group.protocolType().orElse(null))
+                                .setLeader(currentLeader)
+                                .setSkipAssignment(false)
+                                .setErrorCode(errorCode)
+                        );
+
+                    } else if (supportSkippingAssignment) {
+                        boolean isLeader = group.isLeader(newMemberId);
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (isLeader) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        group.completeJoinFuture(member, new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(isLeader)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+
+                    } else {
+                        // Prior to version 9 of the JoinGroup API, we wanted to avoid current leader
+                        // performing trivial assignment while the group is in stable stage, because
+                        // the new assignment in leader's next sync call won't be broadcast by a stable group.
+                        // This could be guaranteed by always returning the old leader id so that the current
+                        // leader won't assume itself as a leader based on the returned message, since the new
+                        // member.id won't match returned leader id, therefore no assignment will be performed.
+
+                        group.completeJoinFuture(member, new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(currentLeader)
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                return maybePrepareRebalance(
+                    group,
+                    "Group's selectedProtocol will change because static member " +
+                        member.memberId() + " with instance id " + groupInstanceId +
+                        " joined with change of protocol; " + "client reason: " + joinReason
+                );
+            }
+
+        } else if (group.isInState(COMPLETING_REBALANCE)) {
+            // if the group is in after-sync stage, upon getting a new join-group of a known static member
+            // we should still trigger a new rebalance, since the old member may already be sent to the leader
+            // for assignment, and hence when the assignment gets back there would be a mismatch of the old member id
+            // with the new replaced member id. As a result the new member id would not get any assignment.
+            return prepareRebalance(group,
+                "Updating metadata for static member " + member.memberId() + " with instance id " +
+                    request.groupInstanceId() + "; client reason: " + joinReason);
+
+        } else if (group.isInState(EMPTY) || group.isInState(DEAD)) {
+            throw new IllegalStateException("Group " + group.groupId() + " was not supposed to be in the state " +
+                group.stateAsString() + " when the unknown static member " + request.groupInstanceId() + " rejoins.");
+
+        } // do nothing if in PREPARING_REBALANCE state.
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Generate a heartbeat key for the timer.
+     *
+     * Package private for testing.
+     *
+     * @param groupId   The group id.
+     * @param memberId  The member id.
+     *
+     * @return the heartbeat key.
+     */
+    static String heartbeatKey(String groupId, String memberId) {
+        return "heartbeat-" + groupId + "-" + memberId;
+    }
+
+    /**
+     * Generate a join key for the timer.
+     *
+     * Package private for testing.
+     *
+     * @param groupId   The group id.
+     *
+     * @return the join key.
+     */
+    static String joinKey(String groupId) {
+        return "join-" + groupId;
+    }
+
+    /**
+     * Generate a sync key for the timer.
+     *
+     * Package private for testing.
+     *
+     * @param groupId   The group id.
+     *
+     * @return the sync key.
+     */
+    static String syncKey(String groupId) {
+        return "sync-" + groupId;
+    }
+
+    /**
+     * Used for testing.
+     *
+     * @return the new member join timeout in milliseconds.
+     */
+    public int genericGroupNewMemberJoinTimeoutMs() {
+        return this.genericGroupNewMemberJoinTimeoutMs;
+    }

Review Comment:
   Could we remove this? `genericGroupNewMemberJoinTimeoutMs` is very likely passed to this object by the test itself so I am not sure to understand why we need to expose it to the test. Is there a reason?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -521,6 +548,7 @@ public void run() {
          */
         @Override
         public void complete(Throwable exception) {
+            CompletableFuture<T> future = result.appendFuture() == null ? this.future : result.appendFuture();

Review Comment:
   I wonder if we should complete both future here. As `scheduleWriteOperation` returns a future, it may be missed used otherwise. What do you think?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java:
##########
@@ -38,6 +38,7 @@
 import org.apache.kafka.server.common.MetadataVersion;
 
 import java.util.ArrayList;
+import java.util.Collections;

Review Comment:
   nit: Could we revert this?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1348 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (group.isNew()) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We set this here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });

Review Comment:
   Don't we need to fail the future if the write fails? Or is it done somewhere else?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -489,8 +513,11 @@ public void run() {
                     // second, then are written to the partition/log, and finally, the response
                     // is put into the deferred event queue.
                     try {
-                        // Apply the records to the state machine.
-                        result.records().forEach(context.coordinator::replay);
+                        // Apply the records to the state machine. If the append-future exists, this
+                        // means that the in-memory state was already updated.
+                        if (result.appendFuture() == null) {
+                            result.records().forEach(context.coordinator::replay);
+                        }

Review Comment:
   How about adding a boolean `replayRecords` to the CoordinatorResult?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1348 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (group.isNew()) {

Review Comment:
   Have you considered checking if the group exists in the map instead of adding this field? Using a field like this has the disadvantage that we must ensure that it is set to false. I think that your implementation already misses it.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/ReplicatedGroupCoordinator.java:
##########
@@ -78,21 +91,56 @@ public CoordinatorBuilder<ReplicatedGroupCoordinator, Record> withSnapshotRegist
             return this;
         }
 
+        @Override
+        public CoordinatorBuilder<ReplicatedGroupCoordinator, Record> withMetadataImage(
+            MetadataImage metadataImage
+        ) {
+            this.metadataImage = metadataImage;
+            return this;
+        }
+
+        @Override
+        public CoordinatorBuilder<ReplicatedGroupCoordinator, Record> withTopicPartition(
+            TopicPartition topicPartition
+        ) {
+            this.topicPartition = topicPartition;
+            return this;
+        }
+
+        @Override
+        public CoordinatorBuilder<ReplicatedGroupCoordinator, Record> withTime(
+            Time time
+        ) {
+            this.time = time;
+            return this;
+        }
+
         @Override
         public ReplicatedGroupCoordinator build() {
             if (logContext == null) logContext = new LogContext();
             if (config == null)
                 throw new IllegalArgumentException("Config must be set.");
             if (snapshotRegistry == null)
                 throw new IllegalArgumentException("SnapshotRegistry must be set.");
+            if (metadataImage == null)
+                throw new IllegalArgumentException("MetadataImage must be set.");
+            if (time == null)

Review Comment:
   `topicPartition` should also be required, I think.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorResult.java:
##########
@@ -37,18 +38,51 @@ public class CoordinatorResult<T, U> {
      */
     private final T response;
 
+    /**
+     * The future to complete once the records are committed.
+     */
+    private final CompletableFuture<T> appendFuture;

Review Comment:
   nit: `completionFuture` or smth similar may be a better name here because we could have an operation without any records.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1348 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (group.isNew()) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We set this here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                newGroupResult = new CoordinatorResult<>(records, appendFuture);
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                CoordinatorResult<Void, Record> joinResult = completeGenericGroupJoin(group);
+                result = result != EMPTY_RESULT ? result : joinResult;

Review Comment:
   I am not satisfied with this logic here. I think that other folks won't understand this... We need to come up with a better way. I will think about it.
   
   I don't recall if I already asked this but would it be possible to push `completeGenericGroupJoin` into `genericGroupJoinNewMember` and `genericGroupJoinExistingMember` instead of having it here?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1362 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,

Review Comment:
   We only write a record when the rebalance completes. This implies that the record is always empty or has members. As you pointed out, a failure happening before the rebalance complete is lost.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }
+
+                result = completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member, group.rebalanceTimeoutMs());
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Void>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if ((member.assignment() == null || member.assignment().length == 0) && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        if (group.isInState(EMPTY) || group.isInState(DEAD)) {
+            // Always accept the request when the group is empty or dead
+            return true;
+
+        } else if (group.isInState(PREPARING_REBALANCE)) {
+            // An existing member is accepted if it is already awaiting. New members are accepted
+            // up to the max group size. Note that the number of awaiting members is used here
+            // for two reasons:
+            // 1) the group size is not reliable as it could already be above the max group size
+            //    if the max group size was reduced.
+            // 2) using the number of awaiting members allows to kick out the last rejoining
+            //    members of the group.
+            return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                group.numAwaitingJoinResponse() < genericGroupMaxSize;
+
+        } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+            // An existing member is accepted. New members are accepted up to the max group size.
+            // Note that the group size is used here. When the group transitions to CompletingRebalance,
+            // members who haven't rejoined are removed.
+            return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+        } else {
+            throw new IllegalStateException("Cannot handle state " + group.stateAsString());
+        }
+    }
+
+    /**
+     * Update a static member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group of the static member.
+     * @param oldMemberId     The existing static member id.
+     * @param newMemberId     The new joining static member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateStaticMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String oldMemberId,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String currentLeader = group.leaderOrNull();
+        GenericGroupMember member = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+
+        // Heartbeat of old member id will expire without effect since the group no longer contains that member id.
+        // New heartbeat shall be scheduled with new member id.
+        rescheduleGenericGroupMemberHeartbeat(group, member);
+
+        int oldRebalanceTimeoutMs = member.rebalanceTimeoutMs();
+        int oldSessionTimeoutMs = member.sessionTimeoutMs();
+        JoinGroupRequestProtocolCollection oldProtocols = member.supportedProtocols();
+
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        if (group.isInState(STABLE)) {
+            // Check if group's selected protocol of next generation will change, if not, simply store group to persist
+            // the updated static member, if yes, rebalance should be triggered to keep the group's assignment
+            // and selected protocol consistent
+            String groupInstanceId = request.groupInstanceId();
+            String selectedProtocolForNextGeneration = group.selectProtocol();
+            if (group.protocolName().orElse("").equals(selectedProtocolForNextGeneration)) {
+                log.info("Static member which joins during Stable stage and doesn't affect " +
+                    "the selected protocol will not trigger a rebalance.");
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    // Starting from version 9 of the JoinGroup API, static members are able to
+                    // skip running the assignor based on the `SkipAssignment` field. We leverage
+                    // this to tell the leader that it is the leader of the group but by skipping
+                    // running the assignor while the group is in stable state.
+                    // Notes:
+                    // 1) This allows the leader to continue monitoring metadata changes for the
+                    // group. Note that any metadata changes happening while the static leader is
+                    // down won't be noticed.
+                    // 2) The assignors are not idempotent nor free from side effects. This is why
+                    // we skip entirely the assignment step as it could generate a different group
+                    // assignment which would be ignored by the group coordinator because the group
+                    // is the stable state.
+                    boolean supportSkippingAssignment = context.apiVersion() >= 9;
+
+                    if (t != null) {
+                        log.warn("Failed to persist metadata for group {}: {}", group.groupId(), t.getMessage());
+
+                        // Failed to persist the member id of the given static member, revert the update of the static member in the group.
+                        group.updateMember(member, oldProtocols, oldRebalanceTimeoutMs, oldSessionTimeoutMs, null);
+                        GenericGroupMember oldMember = group.replaceStaticMember(groupInstanceId, newMemberId, oldMemberId);
+                        rescheduleGenericGroupMemberHeartbeat(group, oldMember);
+
+                        short errorCode = Errors.forException(t).code();
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setMembers(Collections.emptyList())
+                                .setMemberId(UNKNOWN_MEMBER_ID)
+                                .setGenerationId(group.generationId())
+                                .setProtocolName(group.protocolName().orElse(null))
+                                .setProtocolType(group.protocolType().orElse(null))
+                                .setLeader(currentLeader)
+                                .setSkipAssignment(false)
+                                .setErrorCode(errorCode)
+                        );
+
+                    } else if (supportSkippingAssignment) {
+                        boolean isLeader = group.isLeader(newMemberId);
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (isLeader) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        group.completeJoinFuture(member, new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(isLeader)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+
+                    } else {
+                        // Prior to version 9 of the JoinGroup API, we wanted to avoid current leader
+                        // performing trivial assignment while the group is in stable stage, because
+                        // the new assignment in leader's next sync call won't be broadcast by a stable group.
+                        // This could be guaranteed by always returning the old leader id so that the current
+                        // leader won't assume itself as a leader based on the returned message, since the new
+                        // member.id won't match returned leader id, therefore no assignment will be performed.
+
+                        group.completeJoinFuture(member, new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(currentLeader)
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                return maybePrepareRebalance(
+                    group,
+                    "Group's selectedProtocol will change because static member " +
+                        member.memberId() + " with instance id " + groupInstanceId +
+                        " joined with change of protocol; " + "client reason: " + joinReason
+                );
+            }
+
+        } else if (group.isInState(COMPLETING_REBALANCE)) {
+            // if the group is in after-sync stage, upon getting a new join-group of a known static member
+            // we should still trigger a new rebalance, since the old member may already be sent to the leader
+            // for assignment, and hence when the assignment gets back there would be a mismatch of the old member id
+            // with the new replaced member id. As a result the new member id would not get any assignment.
+            return prepareRebalance(group,
+                "Updating metadata for static member " + member.memberId() + " with instance id " +
+                    request.groupInstanceId() + "; client reason: " + joinReason);
+
+        } else if (group.isInState(EMPTY) || group.isInState(DEAD)) {
+            throw new IllegalStateException("Group " + group.groupId() + " was not supposed to be in the state " +
+                group.stateAsString() + " when the unknown static member " + request.groupInstanceId() + " rejoins.");

Review Comment:
   Hum... Those exceptions will be caught by the `CoordinatorWriteEvent` and used to complete the future there. So, I suppose that they will be propagated to the api layer via this mechanism. Do I get this right?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {

Review Comment:
   the existing protocol allows empty group instance ids for static member



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }

Review Comment:
   How about the following? We keep track whether the group was newly created in a boolean. When we get the result from those methods, we check if the group is new, if it is, we check if the result has at least one record. If it does not, we recreate it while adding an empty record for the group.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }
-    }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() {

Review Comment:
   thanks for the catch. will add 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +282,32 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isGroupIdNotEmpty(request.groupId())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleWriteOperation("generic-group-join",
+            topicPartitionFor(request.groupId()),
+            coordinator -> coordinator.genericGroupJoin(context, request, responseFuture)
+        ).exceptionally(exception -> {
+            log.error("Request {} hit an unexpected exception: {}",
+                request, exception.getMessage());

Review Comment:
   this would log all errors while appending/committing and if `generateRecordsAndResponse` throws an unexpected exception. shouldn't we log them? 
   
   it doesn't seem like we do for `consumerGroupHeartbeat()` -- maybe just filter out the coordinator not available / not coordinator error codes?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1230,1358 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = groups.get(groupId) == null;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log

Review Comment:
   simplified the code a bunch. 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] jeffkbkim commented on a diff in pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1221,1331 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We create records here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                newGroupResult = new CoordinatorResult<>(records, appendFuture);
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if the group is new, we need to
+                // persist the group to the timeline map.
+                result = newGroupResult;
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Attempt to complete join group phase. We do not complete
+     * the join group phase if this is the initial rebalance.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteJoinPhase(GenericGroup group) {
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberThenRebalanceOrCompleteJoin(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(group.isLeader(memberId) ?
+                                group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}) with {} members",
+                    groupId, group.generationId(), topicPartition, group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(heartbeatKey(groupId, member.memberId()));
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalanceOrCompleteJoin(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member. Then begin a rebalance or complete the join phase.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateMemberThenRebalanceOrCompleteJoin(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalanceOrCompleteJoin(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.previousState() == EMPTY) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalanceOrCompleteJoin(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state. Otherwise, try
+     * to complete the join phase.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybePrepareRebalanceOrCompleteJoin(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        } else {
+            return tryCompleteJoinPhase(group);
+        }
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     *
+     * Package private for testing.
+     */
+    CoordinatorResult<Void, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.isInState(EMPTY);
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : maybeCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if (!member.hasAssignment() && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<Void, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        switch (group.currentState()) {
+            case EMPTY:
+            case DEAD:
+                // Always accept the request when the group is empty or dead
+                return true;
+            case PREPARING_REBALANCE:
+                // An existing member is accepted if it is already awaiting. New members are accepted
+                // up to the max group size. Note that the number of awaiting members is used here
+                // for two reasons:
+                // 1) the group size is not reliable as it could already be above the max group size
+                //    if the max group size was reduced.
+                // 2) using the number of awaiting members allows to kick out the last rejoining
+                //    members of the group.
+                return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                    group.numAwaitingJoinResponse() < genericGroupMaxSize;
+            case COMPLETING_REBALANCE:
+            case STABLE:
+                // An existing member is accepted. New members are accepted up to the max group size.
+                // Note that the group size is used here. When the group transitions to CompletingRebalance,
+                // members who haven't rejoined are removed.
+                return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+            default:
+                throw new IllegalStateException("Unknown group state: " + group.stateAsString());
+        }
+    }
+
+    /**
+     * Update a static member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group of the static member.
+     * @param oldMemberId     The existing static member id.
+     * @param newMemberId     The new joining static member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateStaticMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String oldMemberId,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String currentLeader = group.leaderOrNull();
+        GenericGroupMember newMember = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+
+        // Heartbeat of old member id will expire without effect since the group no longer contains that member id.
+        // New heartbeat shall be scheduled with new member id.
+        rescheduleGenericGroupMemberHeartbeat(group, newMember);
+
+        int oldRebalanceTimeoutMs = newMember.rebalanceTimeoutMs();
+        int oldSessionTimeoutMs = newMember.sessionTimeoutMs();
+        JoinGroupRequestProtocolCollection oldProtocols = newMember.supportedProtocols();
+
+        group.updateMember(
+            newMember,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        if (group.isInState(STABLE)) {
+            // Check if group's selected protocol of next generation will change, if not, simply store group to persist
+            // the updated static member, if yes, rebalance should be triggered to keep the group's assignment
+            // and selected protocol consistent
+            String groupInstanceId = request.groupInstanceId();
+            String selectedProtocolForNextGeneration = group.selectProtocol();
+            if (group.protocolName().orElse("").equals(selectedProtocolForNextGeneration)) {
+                log.info("Static member which joins during Stable stage and doesn't affect " +
+                    "the selected protocol will not trigger a rebalance.");
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        log.warn("Failed to persist metadata for group {}: {}", group.groupId(), t.getMessage());
+
+                        // Failed to persist the member id of the given static member, revert the update of the static member in the group.
+                        group.updateMember(newMember, oldProtocols, oldRebalanceTimeoutMs, oldSessionTimeoutMs, null);
+                        GenericGroupMember oldMember = group.replaceStaticMember(groupInstanceId, newMemberId, oldMemberId);
+                        rescheduleGenericGroupMemberHeartbeat(group, oldMember);
+
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setMembers(Collections.emptyList())
+                                .setMemberId(UNKNOWN_MEMBER_ID)
+                                .setGenerationId(group.generationId())
+                                .setProtocolName(group.protocolName().orElse(null))
+                                .setProtocolType(group.protocolType().orElse(null))
+                                .setLeader(currentLeader)
+                                .setSkipAssignment(false)
+                                .setErrorCode(Errors.forException(t).code())
+                        );
+                    } else if (context.apiVersion() >= 9) {
+                        // Starting from version 9 of the JoinGroup API, static members are able to
+                        // skip running the assignor based on the `SkipAssignment` field. We leverage
+                        // this to tell the leader that it is the leader of the group but by skipping
+                        // running the assignor while the group is in stable state.
+                        // Notes:
+                        // 1) This allows the leader to continue monitoring metadata changes for the
+                        // group. Note that any metadata changes happening while the static leader is
+                        // down won't be noticed.
+                        // 2) The assignors are not idempotent nor free from side effects. This is why
+                        // we skip entirely the assignment step as it could generate a different group
+                        // assignment which would be ignored by the group coordinator because the group
+                        // is the stable state.
+                        boolean isLeader = group.isLeader(newMemberId);
+
+                        group.completeJoinFuture(newMember, new JoinGroupResponseData()
+                            .setMembers(isLeader ? group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(isLeader)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Prior to version 9 of the JoinGroup API, we wanted to avoid current leader
+                        // performing trivial assignment while the group is in stable stage, because
+                        // the new assignment in leader's next sync call won't be broadcast by a stable group.
+                        // This could be guaranteed by always returning the old leader id so that the current
+                        // leader won't assume itself as a leader based on the returned message, since the new
+                        // member.id won't match returned leader id, therefore no assignment will be performed.
+                        group.completeJoinFuture(newMember, new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(currentLeader)
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+            } else {
+                return maybePrepareRebalanceOrCompleteJoin(
+                    group,
+                    "Group's selectedProtocol will change because static member " +
+                        newMember.memberId() + " with instance id " + groupInstanceId +
+                        " joined with change of protocol; " + "client reason: " + joinReason
+                );
+            }
+        } else if (group.isInState(COMPLETING_REBALANCE)) {
+            // if the group is in after-sync stage, upon getting a new join-group of a known static member
+            // we should still trigger a new rebalance, since the old member may already be sent to the leader
+            // for assignment, and hence when the assignment gets back there would be a mismatch of the old member id
+            // with the new replaced member id. As a result the new member id would not get any assignment.
+            return prepareRebalance(
+                group,
+                "Updating metadata for static member " + newMember.memberId() + " with instance id " +
+                    request.groupInstanceId() + "; client reason: " + joinReason

Review Comment:
   great suggestion. thanks



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }
-    }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());

Review Comment:
   Yeah, we have to stick to the old one 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +282,32 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isGroupIdNotEmpty(request.groupId())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleWriteOperation("generic-group-join",
+            topicPartitionFor(request.groupId()),
+            coordinator -> coordinator.genericGroupJoin(context, request, responseFuture)
+        ).exceptionally(exception -> {
+            log.error("Request {} hit an unexpected exception: {}",
+                request, exception.getMessage());

Review Comment:
   i was thinking about the illegal state exceptions. wouldn't we hide the issue then? maybe we can log only for non api exceptions. wdyt?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -299,10 +299,10 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             topicPartitionFor(request.groupId()),
             coordinator -> coordinator.genericGroupJoin(context, request, responseFuture)
         ).exceptionally(exception -> {
-            log.error("Request {} hit an unexpected exception: {}",
-                request, exception.getMessage());
-
             if (!responseFuture.isDone()) {
+                log.error("Request {} hit an unexpected exception: {}",

Review Comment:
   This would still log in expected cases, no? For instance, when the coordinator for the group is inactive, loading, etc. If you really want to log something, you could perhaps log only if `exception` is not a KafkaException or only when it is a RuntimeException for instance.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -561,7 +625,7 @@ public boolean hasReceivedSyncFromAllMembers() {
      * @return members that have yet to sync.
      */
     public Set<String> allPendingSyncMembers() {
-        return pendingSyncMembers;
+        return new HashSet<>(pendingSyncMembers);

Review Comment:
   thought i addressed this. addressed it now



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1245,4 +1422,1304 @@ public static String consumerGroupSessionTimeoutKey(String groupId, String membe
     public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) {
         return "revocation-timeout-" + groupId + "-" + memberId;
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            removeGroup(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> genericGroup.add(member, null));
+            groups.put(groupId, genericGroup);
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if a group was newly created, we need to append
+                // records to the log to commit the group to the timeline data structure.
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setErrorCode(appendGroupMetadataErrorToResponseError(Errors.forException(t)).code()));
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newEmptyGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Attempt to complete join group phase. We do not complete
+     * the join group phase if this is the initial rebalance.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinPhase(GenericGroup group) {
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                responseFuture
+            );
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (JoinGroupRequest.requiresKnownMemberId(context.apiVersion())) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String genericGroupHeartbeatKey = genericGroupHeartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                genericGroupHeartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberThenRebalanceOrCompleteJoin(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " + group.stateAsString() +
+                            "; client reason: " + JoinGroupRequest.joinReason(request),
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(group.isLeader(memberId) ?
+                                group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(genericGroupJoinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(genericGroupHeartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                genericGroupJoinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}) with {} members",
+                    groupId, group.generationId(), topicPartition, group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(genericGroupHeartbeatKey(groupId, member.memberId()));

Review Comment:
   @dajac i think our last discussion was to also revert this to the existing behavior right? i.e. not implement https://issues.apache.org/jira/browse/KAFKA-13766



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -578,4 +619,19 @@ public void shutdown() {
         Utils.closeQuietly(runtime, "coordinator runtime");
         log.info("Shutdown complete.");
     }
+
+    private boolean isValidGroupId(String groupId, ApiKeys api) {
+        if (api == ApiKeys.OFFSET_COMMIT ||
+            api == ApiKeys.OFFSET_FETCH ||
+            api == ApiKeys.DESCRIBE_GROUPS ||
+            api == ApiKeys.DELETE_GROUPS

Review Comment:
   i will take your suggestion for this PR. however, it does make more sense to have the logic in one place instead of using isGroupIdNotNull/isGroupIdNotEmpty based on the request.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -102,13 +108,34 @@ public Builder withLoader(CoordinatorLoader<Record> loader) {
             return this;
         }
 
+        public Builder withMetadataImage(MetadataImage metadataImage) {
+            this.metadataImage = metadataImage;
+            return this;
+        }
+
+        public Builder withTime(Time time) {
+            this.time = time;
+            return this;
+        }
+
+        public Builder withTimer(Timer timer) {
+            this.timer = timer;
+            return this;
+        }
+
         public GroupCoordinatorService build() {
             if (config == null)
                 throw new IllegalArgumentException("Config must be set.");
             if (writer == null)
                 throw new IllegalArgumentException("Writer must be set.");
             if (loader == null)
                 throw new IllegalArgumentException("Loader must be set.");
+            if (metadataImage == null)
+                throw new IllegalArgumentException("MetadataImage must be set.");
+            if (time == null)
+                throw new IllegalArgumentException("Time must be set.");
+            if (timer == null)

Review Comment:
   i'll remove this for now and add it back if we decide to pass it in later.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -228,14 +261,21 @@ public List<Record> build(TopicsImage topicsImage) {
 
     static class GroupMetadataManagerTestContext {
         static class Builder {
-            final private Time time = new MockTime();
             final private LogContext logContext = new LogContext();
             final private SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
             private MetadataImage metadataImage;
-            private List<PartitionAssignor> assignors;

Review Comment:
   we get illegal state exception if it's not initialized and since it doesn't affect the old protocol i thought it best to initialize it 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));

Review Comment:
   for all tests, we always generate a new record. some tests hide this as it's called in `GroupMetadataManagerContext#joinGenericGroupAsDynamicMember()`.
   
   maybe we can simplify this and just manually create an empty group for all tests except 1 where we test the new record. wdyt?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -228,14 +261,21 @@ public List<Record> build(TopicsImage topicsImage) {
 
     static class GroupMetadataManagerTestContext {
         static class Builder {
-            final private Time time = new MockTime();
             final private LogContext logContext = new LogContext();
             final private SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
             private MetadataImage metadataImage;
-            private List<PartitionAssignor> assignors;

Review Comment:
   we get illegal state exception if it's not initialized and since it doesn't affect the old protocol i thought it best to keep it clean



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2402,143 +2610,2090 @@ public void testOnNewMetadataImage() {
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = genericGroup(context, "group-id");
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = genericGroup(context, "group-id");
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
         }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+
+        // Send as static member join.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setGroupInstanceId("group-instance-id"), true, true, null);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(firstResponse.leader(), firstMemberId);
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, group.allMembers().iterator().next().memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+        assertNull(response.protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertFalse(responseFuture.isDone());
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+        context.timer.advanceClock(2);
+
+        // Advance clock past initial rebalance delay and verify futures are not completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+
+        // Advance clock beyond recomputed delay and make sure the futures have completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldDelayRebalanceUptoRebalanceTimeout() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs + 1);
+
+        CompletableFuture<JoinGroupResponseData> thirdMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        // Advance clock right before rebalance timeout.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+        assertFalse(thirdMemberResponseFuture.isDone());
+
+        // Advance clock beyond rebalance timeout.
+        context.timer.advanceClock(1);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertTrue(thirdMemberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), thirdMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupReplaceStaticMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .build();
+
+
+        // Send join group as static member.
+        CompletableFuture<JoinGroupResponseData> oldMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(oldMemberResponseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Replace static member with new member id. Old member id should be fenced.
+        CompletableFuture<JoinGroupResponseData> newMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        assertFalse(newMemberResponseFuture.isDone());
+        assertTrue(oldMemberResponseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Complete join for new member.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(newMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), newMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemovePendingMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(0, group.size());
+        assertEquals(1, group.numPendingJoinMembers());
+
+        // Advance clock by session timeout. Pending member should be removed from group as heartbeat expires.
+        context.timer.advanceClock(1000);
+        assertEquals(0, group.numPendingJoinMembers());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemoveMember() throws Exception {
+        // Set initial rebalance delay to simulate a long running rebalance.
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupInitialRebalanceDelayMs(10 * 60 * 1000)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        String memberId = group.leaderOrNull();
+        assertEquals(1, group.size());
+
+        // Advance clock by new member join timeout. Member should be removed from group as heartbeat expires.
+        // A group that transitions to Empty after completing join phase will generate records.
+        context.timer.expectResult(heartbeatKey("group-id", memberId), new ExpectedGenericGroupResult(
+            Collections.singletonList(newGroupMetadataRecord("group-id",
+                new GroupMetadataValue()
+                    .setMembers(Collections.emptyList())
+                    .setGeneration(1)
+                    .setLeader(null)
+                    .setProtocolType("consumer")
+                    .setProtocol(null)
+                    .setCurrentStateTimestamp(context.time.milliseconds()),
+                MetadataVersion.latest())),
+            Errors.NONE,
+            false
+        ));
+
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.size());
+    }
+
+    @Test
+    public void testExistingMemberJoinDeadGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertTrue(group.hasMemberId(memberId));
+
+        group.transitionTo(DEAD);
+
+        response = context.joinGenericGroupAsDynamicMember(request.setMemberId(memberId));
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupExistingPendingMemberWithGroupInstanceIdThrowsException() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertTrue(responseFuture.isDone());
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        assertThrows(IllegalStateException.class,
+            () -> context.sendGenericGroupJoin(request.setMemberId(memberId).setGroupInstanceId("group-instance-id"))
+        );
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        GenericGroupMember member = group.member(memberId);
+
+        assertEquals(protocols, member.supportedProtocols());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(1, group.generationId());
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        // Send updated member metadata. This should trigger a rebalance and complete the join phase.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(memberId)
+            .setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+        assertEquals(protocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingLeaderTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertTrue(group.isLeader(memberId));
+        assertEquals(1, group.generationId());
+
+        group.transitionTo(STABLE);
+        // Sending join group as leader should trigger a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with updated metadata. This should trigger a rebalance.
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        JoinGroupRequestData memberRequest = request.setMemberId(memberId).setProtocols(protocols);
+        memberResponseFuture = context.sendGenericGroupJoin(memberRequest);
+
+        assertFalse(memberResponseFuture.isDone());
+
+        // Leader rejoins. This completes the join group phase.
+        leaderResponseFuture = context.sendGenericGroupJoin(request.setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(3, group.generationId());
+        assertEquals(2, group.size());
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberDoesNotTriggerRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with no metadata changes. This does not trigger a rebalance.
+        memberResponseFuture = context.sendGenericGroupJoin(request.setMemberId(memberId));
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, memberResponseFuture.get(5, TimeUnit.SECONDS).generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberInEmptyState() throws Exception {
+        // Existing member joins a group that is in Empty/Dead state. Ask member to rejoin with generation id reset.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(-1, responseFuture.get(5, TimeUnit.SECONDS).generationId());
+    }
+
+    @Test
+    public void testCompleteJoinRemoveNotYetRejoinedDynamicMembers() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.generationId());
+
+        // Add new member. This triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout. This will expire the leader as it has not rejoined.
+        context.timer.advanceClock(1000);
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertTrue(group.hasMemberId(memberResponseFuture.get(5, TimeUnit.SECONDS).memberId()));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseInEmptyStateSkipsRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+        assertEquals(0, group.generationId());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        group.transitionTo(DEAD);
+
+        // Advance clock by initial rebalance delay to complete join phase.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertEquals(0, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseNoMembersRejoinedExtendsJoinPhase() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("first-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(30000)
+            .withRebalanceTimeoutMs(10000)
+            .build();
+
+        // First member joins group and completes join phase.
+        JoinGroupResponseData firstMemberResponse = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), firstMemberResponse.errorCode());
+        String firstMemberId = firstMemberResponse.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        // Second member joins and group goes into rebalancing state.
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setGroupInstanceId("second-instance-id"));
+
+        // First static member rejoins and completes join phase.
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        String secondMemberId = secondMemberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        // Trigger a rebalance. No members rejoined.
+        context.groupMetadataManager.prepareRebalance(group, "trigger rebalance");
+
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(0, group.numAwaitingJoinResponse());
+
+        // Advance clock by rebalance timeout to complete join phase. As long as both members have not
+        // rejoined, we extend the join phase.
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        // Let first and second member rejoin. This should complete the join phase.
+        firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(secondMemberId)
+            .setGroupInstanceId("second-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(3, group.generationId());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testReplaceStaticMemberInStableStateNoError(
+        boolean supportSkippingAssignment
+    ) throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, supportSkippingAssignment);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String oldMemberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. This should update the log with the generated member id.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request
+                .setProtocols(protocols)
+                .setRebalanceTimeoutMs(7000)
+                .setSessionTimeoutMs(4500),
+            true,
+            supportSkippingAssignment,
+            new ExpectedGenericGroupResult(Errors.NONE, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        String newMemberId = group.staticMemberId("group-instance-id");
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMemberId)
+            .setMemberId(newMemberId)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(supportSkippingAssignment)
+            .setErrorCode(Errors.NONE.code());
+
+        if (supportSkippingAssignment) {
+            expectedResponse
+                .setMembers(Collections.singletonList(
+                    new JoinGroupResponseData.JoinGroupResponseMember()
+                        .setMemberId(newMemberId)
+                        .setGroupInstanceId("group-instance-id")
+                        .setMetadata(protocols.find("range").metadata())
+                    ))
+                .setLeader(newMemberId);
+        }
+
+        GenericGroupMember updatedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        assertEquals(newMemberId, updatedMember.memberId());
+        assertEquals(Optional.of("group-instance-id"), updatedMember.groupInstanceId());
+        assertEquals(7000, updatedMember.rebalanceTimeoutMs());
+        assertEquals(4500, updatedMember.sessionTimeoutMs());
+        assertEquals(protocols, updatedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateWithUpdatedProtocolTriggersRebalance() throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. The selected protocol changes and triggers a rebalance.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array())
+        );
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertEquals(2, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateErrors() throws Exception {
+        // If the append future fails, we need to revert the soft state to the original member.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, false);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        GenericGroupMember oldMember = group.member(response.memberId());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID but the append fails. This reverts the soft state of the group.
+        protocols.add(new JoinGroupRequestProtocol()
+                .setName("roundrobin")
+                .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                    Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setSessionTimeoutMs(6000)
+                .setRebalanceTimeoutMs(7000)
+                .setProtocols(protocols),
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMember.memberId())
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code());
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        GenericGroupMember revertedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(oldMember.memberId(), revertedMember.memberId());
+        assertEquals(oldMember.groupInstanceId(), revertedMember.groupInstanceId());
+        assertEquals(oldMember.rebalanceTimeoutMs(), revertedMember.rebalanceTimeoutMs());
+        assertEquals(oldMember.sessionTimeoutMs(), revertedMember.sessionTimeoutMs());
+        assertEquals(oldMember.supportedProtocols(), revertedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInCompletingRebalanceStateTriggersRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID and triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertEquals(2, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+    }
+
+    private <T> void assertUnorderedListEquals(
+        List<T> expected,
+        List<T> actual
+    ) {
+        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
+    }
+
+    private void assertResponseEquals(
+        ConsumerGroupHeartbeatResponseData expected,
+        ConsumerGroupHeartbeatResponseData actual
+    ) {
+        if (!responseEquals(expected, actual)) {
+            assertionFailure()
+                .expected(expected)
+                .actual(actual)
+                .buildAndThrow();
+        }
+    }
+
+    private boolean responseEquals(
+        ConsumerGroupHeartbeatResponseData expected,
+        ConsumerGroupHeartbeatResponseData actual
+    ) {
+        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
+        if (expected.errorCode() != actual.errorCode()) return false;
+        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
+        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
+        if (expected.memberEpoch() != actual.memberEpoch()) return false;
+        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
+        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
+        // Unordered comparison of the assignments.
+        return responseAssignmentEquals(expected.assignment(), actual.assignment());
+    }
+
+    private boolean responseAssignmentEquals(
+        ConsumerGroupHeartbeatResponseData.Assignment expected,
+        ConsumerGroupHeartbeatResponseData.Assignment actual
+    ) {
+        if (expected == actual) return true;
+        if (expected == null) return false;
+        if (actual == null) return false;
+
+        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
+            return false;
+
+        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+    }
+
+    private Map<Uuid, Set<Integer>> fromAssignment(
+        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
+    ) {
+        if (assignment == null) return null;
+
+        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
+        assignment.forEach(topicPartitions -> {
+            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+        });
+        return assignmentMap;
+    }
+
+    private void assertRecordsEquals(
+        List<Record> expectedRecords,
+        List<Record> actualRecords
+    ) {
+        try {
+            assertEquals(expectedRecords.size(), actualRecords.size());
+
+            for (int i = 0; i < expectedRecords.size(); i++) {
+                Record expectedRecord = expectedRecords.get(i);
+                Record actualRecord = actualRecords.get(i);
+                assertRecordEquals(expectedRecord, actualRecord);
+            }
+        } catch (AssertionFailedError e) {
+            assertionFailure()
+                .expected(expectedRecords)
+                .actual(actualRecords)
+                .buildAndThrow();
+        }
+    }
+
+    private void assertRecordEquals(
+        Record expected,
+        Record actual
+    ) {
+        try {
+            assertApiMessageAndVersionEquals(expected.key(), actual.key());
+            assertApiMessageAndVersionEquals(expected.value(), actual.value());
+        } catch (AssertionFailedError e) {
+            assertionFailure()
+                .expected(expected)
+                .actual(actual)
+                .buildAndThrow();
+        }
+    }
+
+    private void assertApiMessageAndVersionEquals(
+        ApiMessageAndVersion expected,
+        ApiMessageAndVersion actual
+    ) {
+        if (expected == actual) return;
+
+        assertEquals(expected.version(), actual.version());
+
+        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
+            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
+            // always guaranteed. Therefore, we need a special comparator.
+            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
+                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
+            ConsumerGroupCurrentMemberAssignmentValue actualValue =
+                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+
+            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
+            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
+            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
+            assertEquals(expectedValue.error(), actualValue.error());
+            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
+            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+
+            // We transform those to Maps before comparing them.
+            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
+                fromTopicPartitions(actualValue.assignedPartitions()));
+            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
+                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
+            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
+                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
+        } else {
+            assertEquals(expected.message(), actual.message());
+        }
+    }
+
+    private Map<Uuid, Set<Integer>> fromTopicPartitions(
+        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
+    ) {
+        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
+        assignment.forEach(topicPartitions -> {
+            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
         });
         return assignmentMap;
     }
+
+    private static GenericGroup genericGroup(GroupMetadataManagerTestContext context, String groupId) {
+        return context.groupMetadataManager.getOrMaybeCreateGenericGroup(groupId, false);
+    }
+
+    private List<String> verifyGenericGroupJoinResponses(
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures,
+        int expectedSuccessCount,
+        Errors expectedFailure
+    ) {
+        int successCount = 0;
+        List<String> memberIds = new ArrayList<>();
+        for (CompletableFuture<JoinGroupResponseData> responseFuture : responseFutures) {
+            if (!responseFuture.isDone()) {
+                fail("All responseFutures should be completed.");
+            }
+            try {
+                if (responseFuture.get(5, TimeUnit.SECONDS).errorCode() == Errors.NONE.code()) {
+                    successCount++;
+                } else {
+                    assertEquals(
+                        expectedFailure.code(),
+                        responseFuture.get(5, TimeUnit.SECONDS).errorCode()
+                    );
+                }
+                memberIds.add(responseFuture.get(5, TimeUnit.SECONDS).memberId());
+            } catch (Exception e) {
+                fail("Unexpected exception: " + e.getMessage());
+            }
+        }
+
+        assertEquals(expectedSuccessCount, successCount);
+        return memberIds;
+    }
+
+    /**
+     * Verify the records that should be appended and complete the append future based on a
+     * configured error. Run any assertions to verify the result of the future completion.
+     *
+     * @param expectedResult  The expected result to compare against.
+     * @param result          The result from expiring a join/heartbeat/sync operation.
+     */
+    private static void verifyCoordinatorResult(
+        ExpectedGenericGroupResult expectedResult,
+        CoordinatorResult<Void, Record> result
+    ) {
+        if (expectedResult == null) {
+            assertEquals(EMPTY_RESULT, result);
+        } else {
+            assertEquals(expectedResult.records, result.records());
+            if (expectedResult.mockError == Errors.NONE) {
+                result.appendFuture().complete(null);
+            } else {
+                result.appendFuture().completeExceptionally(expectedResult.mockError.exception());
+            }

Review Comment:
   removed this method, and now individual tests do the validation. 
   
   one exception is for timer operation expirations - as the majority of the cases will not result in any records, i have done the validation inside MockCoordinatorTimer.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1362 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (group.isNew()) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                result = new CoordinatorResult<>(records, appendFuture);
+                genericGroupJoinMember(context, request, group, isUnknownMember, responseFuture);
+            } else {
+                result = genericGroupJoinMember(context, request, group, isUnknownMember, responseFuture);
+            }
+        }
+        return result;
+    }
+
+    private CoordinatorResult<Void, Record> genericGroupJoinMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        boolean isUnknownMember,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+        String joinReason = request.reason();
+        String memberId = request.memberId();
+        if (joinReason == null || joinReason.isEmpty()) {
+            joinReason = "not provided";
+        }
+
+        if (!acceptJoiningMember(group, memberId)) {
+            group.remove(memberId);
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+            );
+
+        } else if (isUnknownMember) {
+            result = genericGroupJoinNewMember(
+                context,
+                request,
+                group,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            result = genericGroupJoinExistingMember(
+                context,
+                request,
+                group,
+                joinReason,
+                responseFuture
+            );
+        }
+
+        tryCompleteJoin(group);
+        return result;
+    }
+
+    private CoordinatorResult<Void, Record> tryCompleteJoin(
+        GenericGroup group
+    ) {
+        // Attempt to complete join group phase. We do not complete
+        // the join group phase if this is the initial rebalance.
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.generationId() != 0
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);

Review Comment:
   which illegal state exception are you referring 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] jeffkbkim commented on a diff in pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }
+
+                result = completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member, group.rebalanceTimeoutMs());
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Void>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if ((member.assignment() == null || member.assignment().length == 0) && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        if (group.isInState(EMPTY) || group.isInState(DEAD)) {
+            // Always accept the request when the group is empty or dead
+            return true;
+
+        } else if (group.isInState(PREPARING_REBALANCE)) {
+            // An existing member is accepted if it is already awaiting. New members are accepted
+            // up to the max group size. Note that the number of awaiting members is used here
+            // for two reasons:
+            // 1) the group size is not reliable as it could already be above the max group size
+            //    if the max group size was reduced.
+            // 2) using the number of awaiting members allows to kick out the last rejoining
+            //    members of the group.
+            return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                group.numAwaitingJoinResponse() < genericGroupMaxSize;
+
+        } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+            // An existing member is accepted. New members are accepted up to the max group size.
+            // Note that the group size is used here. When the group transitions to CompletingRebalance,
+            // members who haven't rejoined are removed.
+            return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+        } else {
+            throw new IllegalStateException("Cannot handle state " + group.stateAsString());
+        }
+    }
+
+    /**
+     * Update a static member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group of the static member.
+     * @param oldMemberId     The existing static member id.
+     * @param newMemberId     The new joining static member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateStaticMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String oldMemberId,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String currentLeader = group.leaderOrNull();
+        GenericGroupMember member = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+
+        // Heartbeat of old member id will expire without effect since the group no longer contains that member id.
+        // New heartbeat shall be scheduled with new member id.
+        rescheduleGenericGroupMemberHeartbeat(group, member);
+
+        int oldRebalanceTimeoutMs = member.rebalanceTimeoutMs();
+        int oldSessionTimeoutMs = member.sessionTimeoutMs();
+        JoinGroupRequestProtocolCollection oldProtocols = member.supportedProtocols();
+
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        if (group.isInState(STABLE)) {
+            // Check if group's selected protocol of next generation will change, if not, simply store group to persist
+            // the updated static member, if yes, rebalance should be triggered to keep the group's assignment
+            // and selected protocol consistent
+            String groupInstanceId = request.groupInstanceId();
+            String selectedProtocolForNextGeneration = group.selectProtocol();
+            if (group.protocolName().orElse("").equals(selectedProtocolForNextGeneration)) {
+                log.info("Static member which joins during Stable stage and doesn't affect " +
+                    "the selected protocol will not trigger a rebalance.");
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    // Starting from version 9 of the JoinGroup API, static members are able to
+                    // skip running the assignor based on the `SkipAssignment` field. We leverage
+                    // this to tell the leader that it is the leader of the group but by skipping
+                    // running the assignor while the group is in stable state.
+                    // Notes:
+                    // 1) This allows the leader to continue monitoring metadata changes for the
+                    // group. Note that any metadata changes happening while the static leader is
+                    // down won't be noticed.
+                    // 2) The assignors are not idempotent nor free from side effects. This is why
+                    // we skip entirely the assignment step as it could generate a different group
+                    // assignment which would be ignored by the group coordinator because the group
+                    // is the stable state.
+                    boolean supportSkippingAssignment = context.apiVersion() >= 9;
+
+                    if (t != null) {
+                        log.warn("Failed to persist metadata for group {}: {}", group.groupId(), t.getMessage());
+
+                        // Failed to persist the member id of the given static member, revert the update of the static member in the group.
+                        group.updateMember(member, oldProtocols, oldRebalanceTimeoutMs, oldSessionTimeoutMs, null);
+                        GenericGroupMember oldMember = group.replaceStaticMember(groupInstanceId, newMemberId, oldMemberId);
+                        rescheduleGenericGroupMemberHeartbeat(group, oldMember);
+
+                        short errorCode = Errors.forException(t).code();
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setMembers(Collections.emptyList())
+                                .setMemberId(UNKNOWN_MEMBER_ID)
+                                .setGenerationId(group.generationId())
+                                .setProtocolName(group.protocolName().orElse(null))
+                                .setProtocolType(group.protocolType().orElse(null))
+                                .setLeader(currentLeader)
+                                .setSkipAssignment(false)
+                                .setErrorCode(errorCode)
+                        );
+
+                    } else if (supportSkippingAssignment) {
+                        boolean isLeader = group.isLeader(newMemberId);
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (isLeader) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        group.completeJoinFuture(member, new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(isLeader)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+
+                    } else {
+                        // Prior to version 9 of the JoinGroup API, we wanted to avoid current leader
+                        // performing trivial assignment while the group is in stable stage, because
+                        // the new assignment in leader's next sync call won't be broadcast by a stable group.
+                        // This could be guaranteed by always returning the old leader id so that the current
+                        // leader won't assume itself as a leader based on the returned message, since the new
+                        // member.id won't match returned leader id, therefore no assignment will be performed.
+
+                        group.completeJoinFuture(member, new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(currentLeader)
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                return maybePrepareRebalance(
+                    group,
+                    "Group's selectedProtocol will change because static member " +
+                        member.memberId() + " with instance id " + groupInstanceId +
+                        " joined with change of protocol; " + "client reason: " + joinReason
+                );
+            }
+
+        } else if (group.isInState(COMPLETING_REBALANCE)) {
+            // if the group is in after-sync stage, upon getting a new join-group of a known static member
+            // we should still trigger a new rebalance, since the old member may already be sent to the leader
+            // for assignment, and hence when the assignment gets back there would be a mismatch of the old member id
+            // with the new replaced member id. As a result the new member id would not get any assignment.
+            return prepareRebalance(group,
+                "Updating metadata for static member " + member.memberId() + " with instance id " +
+                    request.groupInstanceId() + "; client reason: " + joinReason);
+
+        } else if (group.isInState(EMPTY) || group.isInState(DEAD)) {
+            throw new IllegalStateException("Group " + group.groupId() + " was not supposed to be in the state " +
+                group.stateAsString() + " when the unknown static member " + request.groupInstanceId() + " rejoins.");

Review Comment:
   The write event will catch the exception and complete the event's future. I added a handler to GroupCoordinatorService.java for these unexpected exceptions.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }
+
+                result = completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member, group.rebalanceTimeoutMs());
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Void>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;

Review Comment:
   added a previousState to GenericGroup. we will rely on this instead to confirm that a group is undergoing an initial rebalance (previous state == empty)



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -521,6 +548,7 @@ public void run() {
          */
         @Override
         public void complete(Throwable exception) {
+            CompletableFuture<T> future = result.appendFuture() == null ? this.future : result.appendFuture();

Review Comment:
   as discussed offline, we will complete both futures. the append future will be completed first and the event future will complete the join response if it's not already completed.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1243,4 +1406,1283 @@ public static String consumerGroupSessionTimeoutKey(String groupId, String membe
     public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) {
         return "revocation-timeout-" + groupId + "-" + memberId;
     }
+
+     /** Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if a group was newly created, we need to append
+                // records to the log to commit the group to the timeline data structure.
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newEmptyGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Attempt to complete join group phase. We do not complete
+     * the join group phase if this is the initial rebalance.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinPhase(GenericGroup group) {
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                responseFuture
+            );
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (JoinGroupRequest.requiresKnownMemberId(context.apiVersion())) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberThenRebalanceOrCompleteJoin(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " + group.stateAsString() +
+                            "; client reason: " + JoinGroupRequest.joinReason(request),
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(group.isLeader(memberId) ?
+                                group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}) with {} members",
+                    groupId, group.generationId(), topicPartition, group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(heartbeatKey(groupId, member.memberId()));
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            false,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalanceOrCompleteJoin(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member. Then begin a rebalance or complete the join phase.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateMemberThenRebalanceOrCompleteJoin(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalanceOrCompleteJoin(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.previousState() == EMPTY) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalanceOrCompleteJoin(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + JoinGroupRequest.joinReason(request));
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state. Otherwise, try
+     * to complete the join phase.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybePrepareRebalanceOrCompleteJoin(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        } else {
+            return maybeCompleteJoinPhase(group);
+        }
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     *
+     * Package private for testing.
+     */
+    CoordinatorResult<Void, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.isInState(EMPTY);
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : maybeCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if (!member.hasAssignment() && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            false,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<Void, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {

Review Comment:
   updated and 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] dajac commented on a diff in pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -3022,143 +3242,2087 @@ public void testOnLoaded() {
         assertNotNull(context.timer.timeout(consumerGroupRevocationTimeoutKey("foo", "foo-1")));
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
-
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
-        }
-    }
+    @Test
+    public void testGenerateRecordsOnNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get().errorCode());
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        GenericGroup group = context.createGenericGroup("group-id");
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        assertEquals(
+            Collections.singletonList(RecordHelpers.newEmptyGroupMetadataRecord(group, MetadataVersion.latest())),
+            result.records()
+        );
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
+        context.createGenericGroup("group-id");
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        IntStream.range(0, 10).forEach(i -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+            assertFalse(responseFuture.isDone());
+            assertTrue(result.records().isEmpty());
         });
-        return assignmentMap;
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        GenericGroup group = context.createGenericGroup("group-id");
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> firstRoundFutures = new ArrayList<>();
+        IntStream.range(0, groupMaxSize + 1).forEach(i -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            firstRoundFutures.add(responseFuture);
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture, requiredKnownMemberId);
+            assertTrue(responseFuture.isDone());
+            try {
+                assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get().errorCode());
+            } catch (Exception ignored) {

Review Comment:
   gotcha. it is fine like 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1243,4 +1406,1283 @@ public static String consumerGroupSessionTimeoutKey(String groupId, String membe
     public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) {
         return "revocation-timeout-" + groupId + "-" + memberId;
     }
+
+     /** Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if a group was newly created, we need to append
+                // records to the log to commit the group to the timeline data structure.
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());

Review Comment:
   then do you think we can move `appendGroupMetadataErrorToResponseError` back to GroupMetadataManager?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }
+
+                result = completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member, group.rebalanceTimeoutMs());
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Void>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;

Review Comment:
   I thought the "initial" rebalance only applied to when the group is first created from https://github.com/apache/kafka/pull/2758 but according to https://cwiki.apache.org/confluence/display/KAFKA/KIP-134%3A+Delay+initial+consumer+group+rebalance it looks like we want this for an empty group as well
   
   > Adding a configurable delay to the initial rebalance of a new, or empty, consumer group [...]
   
   So, i will revert this change.
   
   However, it's awkward because here we consider "initial rebalance" to be an empty group. But a different part of the code checks generation id == 0. maybe it's because we don't know the previous state
   
   from GroupCoordinator#addMemberAndRebalance:
   ```
       // update the newMemberAdded flag to indicate that the join group can be further delayed
       if (group.is(PreparingRebalance) && group.generationId == 0)
         group.newMemberAdded = true
   ```
   
   I think we need to revert the change and include `initialRebalanceDelayMs` to the GenericGroup object and rely on that to check whether the group is undergoing an initial rebalance. 



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -171,70 +260,152 @@ GroupMetadataManager build() {
     /**
      * The maximum number of members allowed in a single consumer group.
      */
-    private final int consumerGroupMaxSize;
+    private final int groupMaxSize;
 
     /**
      * The heartbeat interval for consumer groups.
      */
     private final int consumerGroupHeartbeatIntervalMs;
 
     /**
-     * The topics metadata (or image).
+     * The metadata image.
+     */
+    private MetadataImage metadataImage;
+
+    // Rest of the fields are used for the generic group APIs.
+
+    /**
+     * An empty result returned to the state machine. This means that
+     * there are no records to append to the log.
+     *
+     * Package private for testing.
+     */
+    static final CoordinatorResult<CompletableFuture<Errors>, Record> EMPTY_RESULT =
+        new CoordinatorResult<>(Collections.emptyList(), CompletableFuture.completedFuture(null));
+
+    /**
+     * Initial rebalance delay for members joining a generic group.
+     */
+    private final int initialRebalanceDelayMs;
+
+    /**
+     * The timeout used to wait for a new member in milliseconds.
+     */
+    private final int newMemberJoinTimeoutMs;
+
+    /**
+     * The group minimum session timeout.
+     */
+    private final int groupMinSessionTimeoutMs;
+
+    /**
+     * The group maximum session timeout.
+     */
+    private final int groupMaxSessionTimeoutMs;
+
+    /**
+     * The timer to add and cancel group operations.
      */
-    private TopicsImage topicsImage;
+    private final Timer<CompletableFuture<Errors>, Record> timer;
+
+    /**
+     * The time.
+     */
+    private final Time time;
 
     private GroupMetadataManager(
         SnapshotRegistry snapshotRegistry,
         LogContext logContext,
         List<PartitionAssignor> assignors,
-        TopicsImage topicsImage,
-        int consumerGroupMaxSize,
-        int consumerGroupHeartbeatIntervalMs
+        MetadataImage metadataImage,
+        TopicPartition topicPartition,
+        int groupMaxSize,
+        int consumerGroupHeartbeatIntervalMs,
+        int initialRebalanceDelayMs,
+        int newMemberJoinTimeoutMs,
+        int groupMinSessionTimeoutMs,
+        int groupMaxSessionTimeoutMs,
+        Timer<CompletableFuture<Errors>, Record> timer,
+        Time time
     ) {
+        this.logContext = logContext;
         this.log = logContext.logger(GroupMetadataManager.class);
         this.snapshotRegistry = snapshotRegistry;
-        this.topicsImage = topicsImage;
+        this.metadataImage = metadataImage;
         this.assignors = assignors.stream().collect(Collectors.toMap(PartitionAssignor::name, Function.identity()));
+        this.topicPartition = topicPartition;
         this.defaultAssignor = assignors.get(0);
         this.groups = new TimelineHashMap<>(snapshotRegistry, 0);
-        this.consumerGroupMaxSize = consumerGroupMaxSize;
+        this.groupMaxSize = groupMaxSize;
         this.consumerGroupHeartbeatIntervalMs = consumerGroupHeartbeatIntervalMs;
+        this.initialRebalanceDelayMs = initialRebalanceDelayMs;
+        this.newMemberJoinTimeoutMs = newMemberJoinTimeoutMs;
+        this.groupMinSessionTimeoutMs = groupMinSessionTimeoutMs;
+        this.groupMaxSessionTimeoutMs = groupMaxSessionTimeoutMs;
+        this.timer = timer;
+        this.time = time;
+    }
+
+    /**
+     * When a new metadata image is pushed.
+     *
+     * @param metadataImage The new metadata image.
+     */
+    public void onNewMetadataImage(MetadataImage metadataImage) {
+        this.metadataImage = metadataImage;
     }
 
     /**
      * Gets or maybe creates a consumer group.
      *
      * @param groupId           The group id.
+     * @param groupType         The group type (generic or consumer).
      * @param createIfNotExists A boolean indicating whether the group should be
      *                          created if it does not exist.
      *
      * @return A ConsumerGroup.
+     * @throws InvalidGroupIdException  if the group id is invalid.
      * @throws GroupIdNotFoundException if the group does not exist and createIfNotExists is false or
      *                                  if the group is not a consumer group.
      *
      * Package private for testing.
      */
-    ConsumerGroup getOrMaybeCreateConsumerGroup(
+    // Package private for testing.
+    Group getOrMaybeCreateGroup(

Review Comment:
   i am not convinced. the downside is that it will be harder to guarantee to uniqueness of the group id. it also means that we would have to check both maps for all other operations (e.g. list, delete, etc.). i think that it would be better to keep them in a single map.
   
   for this particular case, we could just have two methods:  `getOrMaybeCreateConsumerGroup` and `getOrMaybeCreateGenericGroup`.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -171,70 +260,152 @@ GroupMetadataManager build() {
     /**
      * The maximum number of members allowed in a single consumer group.
      */
-    private final int consumerGroupMaxSize;
+    private final int groupMaxSize;
 
     /**
      * The heartbeat interval for consumer groups.
      */
     private final int consumerGroupHeartbeatIntervalMs;
 
     /**
-     * The topics metadata (or image).
+     * The metadata image.
+     */
+    private MetadataImage metadataImage;
+
+    // Rest of the fields are used for the generic group APIs.
+
+    /**
+     * An empty result returned to the state machine. This means that
+     * there are no records to append to the log.
+     *
+     * Package private for testing.
+     */
+    static final CoordinatorResult<CompletableFuture<Errors>, Record> EMPTY_RESULT =
+        new CoordinatorResult<>(Collections.emptyList(), CompletableFuture.completedFuture(null));
+
+    /**
+     * Initial rebalance delay for members joining a generic group.
+     */
+    private final int initialRebalanceDelayMs;
+
+    /**
+     * The timeout used to wait for a new member in milliseconds.
+     */
+    private final int newMemberJoinTimeoutMs;
+
+    /**
+     * The group minimum session timeout.
+     */
+    private final int groupMinSessionTimeoutMs;
+
+    /**
+     * The group maximum session timeout.
+     */
+    private final int groupMaxSessionTimeoutMs;
+
+    /**
+     * The timer to add and cancel group operations.
      */
-    private TopicsImage topicsImage;
+    private final Timer<CompletableFuture<Errors>, Record> timer;
+
+    /**
+     * The time.
+     */
+    private final Time time;
 
     private GroupMetadataManager(
         SnapshotRegistry snapshotRegistry,
         LogContext logContext,
         List<PartitionAssignor> assignors,
-        TopicsImage topicsImage,
-        int consumerGroupMaxSize,
-        int consumerGroupHeartbeatIntervalMs
+        MetadataImage metadataImage,
+        TopicPartition topicPartition,
+        int groupMaxSize,
+        int consumerGroupHeartbeatIntervalMs,
+        int initialRebalanceDelayMs,
+        int newMemberJoinTimeoutMs,
+        int groupMinSessionTimeoutMs,
+        int groupMaxSessionTimeoutMs,
+        Timer<CompletableFuture<Errors>, Record> timer,
+        Time time
     ) {
+        this.logContext = logContext;
         this.log = logContext.logger(GroupMetadataManager.class);
         this.snapshotRegistry = snapshotRegistry;
-        this.topicsImage = topicsImage;
+        this.metadataImage = metadataImage;
         this.assignors = assignors.stream().collect(Collectors.toMap(PartitionAssignor::name, Function.identity()));
+        this.topicPartition = topicPartition;
         this.defaultAssignor = assignors.get(0);
         this.groups = new TimelineHashMap<>(snapshotRegistry, 0);
-        this.consumerGroupMaxSize = consumerGroupMaxSize;
+        this.groupMaxSize = groupMaxSize;
         this.consumerGroupHeartbeatIntervalMs = consumerGroupHeartbeatIntervalMs;
+        this.initialRebalanceDelayMs = initialRebalanceDelayMs;
+        this.newMemberJoinTimeoutMs = newMemberJoinTimeoutMs;
+        this.groupMinSessionTimeoutMs = groupMinSessionTimeoutMs;
+        this.groupMaxSessionTimeoutMs = groupMaxSessionTimeoutMs;
+        this.timer = timer;
+        this.time = time;
+    }
+
+    /**
+     * When a new metadata image is pushed.
+     *
+     * @param metadataImage The new metadata image.
+     */
+    public void onNewMetadataImage(MetadataImage metadataImage) {
+        this.metadataImage = metadataImage;
     }
 
     /**
      * Gets or maybe creates a consumer group.
      *
      * @param groupId           The group id.
+     * @param groupType         The group type (generic or consumer).
      * @param createIfNotExists A boolean indicating whether the group should be
      *                          created if it does not exist.
      *
      * @return A ConsumerGroup.
+     * @throws InvalidGroupIdException  if the group id is invalid.
      * @throws GroupIdNotFoundException if the group does not exist and createIfNotExists is false or
      *                                  if the group is not a consumer group.
      *
      * Package private for testing.
      */
-    ConsumerGroup getOrMaybeCreateConsumerGroup(
+    // Package private for testing.
+    Group getOrMaybeCreateGroup(
         String groupId,
+        Group.GroupType groupType,
         boolean createIfNotExists
-    ) throws GroupIdNotFoundException {
+    ) throws InvalidGroupIdException, GroupIdNotFoundException {
+        if (groupId == null || groupId.isEmpty()) {
+            throw new InvalidGroupIdException(String.format("Group id %s is invalid.", groupId));
+        }

Review Comment:
   i removed this; we have request validation in GroupCoordinatorService#joinGroup. what do you think of having all request validation there?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata())).collect(Collectors.toList());
+
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(protocols))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(protocols)) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(protocols)) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            return tryCompleteJoinElseSchedule(group);
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Errors> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((error, t) -> {
+                    if (t != null || error != Errors.NONE) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        String message = t != null ? t.getMessage() : error.message();
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), message);
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member);
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {

Review Comment:
   if the member is awaiting on a join/sync response then we can't remove the member on hb expiration right?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +282,32 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isGroupIdNotEmpty(request.groupId())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleWriteOperation("generic-group-join",
+            topicPartitionFor(request.groupId()),
+            coordinator -> coordinator.genericGroupJoin(context, request, responseFuture)
+        ).exceptionally(exception -> {
+            log.error("Request {} hit an unexpected exception: {}",
+                request, exception.getMessage());

Review Comment:
   I wonder if we should remove this because it will log all errors now.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +282,32 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isGroupIdNotEmpty(request.groupId())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleWriteOperation("generic-group-join",
+            topicPartitionFor(request.groupId()),
+            coordinator -> coordinator.genericGroupJoin(context, request, responseFuture)
+        ).exceptionally(exception -> {
+            log.error("Request {} hit an unexpected exception: {}",
+                request, exception.getMessage());
+
+            if (!responseFuture.isDone()) {
+                Errors clientError = toResponseError(Errors.forException(exception));

Review Comment:
   I am a bit confused here. Don't we need to apply this conversion to `responseFuture` as well?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -579,4 +618,32 @@ public void shutdown() {
         Utils.closeQuietly(runtime, "coordinator runtime");
         log.info("Shutdown complete.");
     }
+
+    private static boolean isGroupIdNotEmpty(String groupId) {
+        return groupId != null && !groupId.isEmpty();
+    }
+
+    private static Errors toResponseError(Errors appendError) {
+        switch (appendError) {
+            case UNKNOWN_TOPIC_OR_PARTITION:
+            case NOT_ENOUGH_REPLICAS:
+            case NOT_ENOUGH_REPLICAS_AFTER_APPEND:

Review Comment:
   You can remove this one because it can't happen now.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -264,7 +403,7 @@ public MetadataImage image() {
     ConsumerGroup getOrMaybeCreateConsumerGroup(
         String groupId,
         boolean createIfNotExists
-    ) throws GroupIdNotFoundException {
+    ) {

Review Comment:
   Could we bring this back?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -579,4 +618,32 @@ public void shutdown() {
         Utils.closeQuietly(runtime, "coordinator runtime");
         log.info("Shutdown complete.");
     }
+
+    private static boolean isGroupIdNotEmpty(String groupId) {
+        return groupId != null && !groupId.isEmpty();
+    }
+
+    private static Errors toResponseError(Errors appendError) {
+        switch (appendError) {
+            case UNKNOWN_TOPIC_OR_PARTITION:
+            case NOT_ENOUGH_REPLICAS:
+            case NOT_ENOUGH_REPLICAS_AFTER_APPEND:
+                return COORDINATOR_NOT_AVAILABLE;
+
+            case NOT_LEADER_OR_FOLLOWER:
+            case KAFKA_STORAGE_ERROR:
+                return NOT_COORDINATOR;
+
+            case REQUEST_TIMED_OUT:

Review Comment:
   ditto.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1221,1331 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We create records here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());

Review Comment:
   For my understanding, we don't fail the future here because the event future will do it. Am I correct?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1221,1331 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We create records here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                newGroupResult = new CoordinatorResult<>(records, appendFuture);
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if the group is new, we need to
+                // persist the group to the timeline map.
+                result = newGroupResult;
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Attempt to complete join group phase. We do not complete
+     * the join group phase if this is the initial rebalance.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteJoinPhase(GenericGroup group) {
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;

Review Comment:
   nit: I was considering whether we should have an helper in the JoinRequest class for this. It could be something like `JoinRequest#requireKnownMemberId(short version)`. That advantage is that it would centralize all the version handling in one place. We could do the same for the other similar cases. What do you think?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1221,1331 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",

Review Comment:
   I think that we should rather add this to the `onLoaded` method rather than here. The issue is that it will also log all the non-compacted records and that will be misleading.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1221,1331 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We create records here
+                // while the group is still empty.
+

Review Comment:
   nit: remove empty line.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1230,1358 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = groups.get(groupId) == null;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log

Review Comment:
   I see. I wonder if we could have a `newGroupEmptyMetadataRecord` which generate an empty record for the group in this case to avoid this issue. Would this work? I am asking because I think that centralising would simplify the code.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -579,4 +618,32 @@ public void shutdown() {
         Utils.closeQuietly(runtime, "coordinator runtime");
         log.info("Shutdown complete.");
     }
+
+    private static boolean isGroupIdNotEmpty(String groupId) {
+        return groupId != null && !groupId.isEmpty();
+    }
+
+    private static Errors toResponseError(Errors appendError) {

Review Comment:
   It may be better to inline this code because the handling could be different depending on the request type. If I remember correctly, it is slightly different for offset commits for instance.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1221,1331 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We create records here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                newGroupResult = new CoordinatorResult<>(records, appendFuture);
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if the group is new, we need to
+                // persist the group to the timeline map.
+                result = newGroupResult;
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Attempt to complete join group phase. We do not complete
+     * the join group phase if this is the initial rebalance.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteJoinPhase(GenericGroup group) {
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberThenRebalanceOrCompleteJoin(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(group.isLeader(memberId) ?
+                                group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}) with {} members",
+                    groupId, group.generationId(), topicPartition, group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(heartbeatKey(groupId, member.memberId()));
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalanceOrCompleteJoin(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member. Then begin a rebalance or complete the join phase.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateMemberThenRebalanceOrCompleteJoin(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalanceOrCompleteJoin(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.previousState() == EMPTY) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalanceOrCompleteJoin(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state. Otherwise, try
+     * to complete the join phase.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybePrepareRebalanceOrCompleteJoin(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        } else {
+            return tryCompleteJoinPhase(group);
+        }
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     *
+     * Package private for testing.
+     */
+    CoordinatorResult<Void, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.isInState(EMPTY);
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : maybeCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if (!member.hasAssignment() && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<Void, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        switch (group.currentState()) {
+            case EMPTY:
+            case DEAD:
+                // Always accept the request when the group is empty or dead
+                return true;
+            case PREPARING_REBALANCE:
+                // An existing member is accepted if it is already awaiting. New members are accepted
+                // up to the max group size. Note that the number of awaiting members is used here
+                // for two reasons:
+                // 1) the group size is not reliable as it could already be above the max group size
+                //    if the max group size was reduced.
+                // 2) using the number of awaiting members allows to kick out the last rejoining
+                //    members of the group.
+                return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                    group.numAwaitingJoinResponse() < genericGroupMaxSize;
+            case COMPLETING_REBALANCE:
+            case STABLE:
+                // An existing member is accepted. New members are accepted up to the max group size.
+                // Note that the group size is used here. When the group transitions to CompletingRebalance,
+                // members who haven't rejoined are removed.
+                return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+            default:
+                throw new IllegalStateException("Unknown group state: " + group.stateAsString());
+        }
+    }
+
+    /**
+     * Update a static member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group of the static member.
+     * @param oldMemberId     The existing static member id.
+     * @param newMemberId     The new joining static member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateStaticMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String oldMemberId,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String currentLeader = group.leaderOrNull();
+        GenericGroupMember newMember = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+
+        // Heartbeat of old member id will expire without effect since the group no longer contains that member id.
+        // New heartbeat shall be scheduled with new member id.
+        rescheduleGenericGroupMemberHeartbeat(group, newMember);
+
+        int oldRebalanceTimeoutMs = newMember.rebalanceTimeoutMs();
+        int oldSessionTimeoutMs = newMember.sessionTimeoutMs();
+        JoinGroupRequestProtocolCollection oldProtocols = newMember.supportedProtocols();
+
+        group.updateMember(
+            newMember,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        if (group.isInState(STABLE)) {
+            // Check if group's selected protocol of next generation will change, if not, simply store group to persist
+            // the updated static member, if yes, rebalance should be triggered to keep the group's assignment
+            // and selected protocol consistent
+            String groupInstanceId = request.groupInstanceId();
+            String selectedProtocolForNextGeneration = group.selectProtocol();
+            if (group.protocolName().orElse("").equals(selectedProtocolForNextGeneration)) {
+                log.info("Static member which joins during Stable stage and doesn't affect " +
+                    "the selected protocol will not trigger a rebalance.");
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        log.warn("Failed to persist metadata for group {}: {}", group.groupId(), t.getMessage());
+
+                        // Failed to persist the member id of the given static member, revert the update of the static member in the group.
+                        group.updateMember(newMember, oldProtocols, oldRebalanceTimeoutMs, oldSessionTimeoutMs, null);
+                        GenericGroupMember oldMember = group.replaceStaticMember(groupInstanceId, newMemberId, oldMemberId);
+                        rescheduleGenericGroupMemberHeartbeat(group, oldMember);
+
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setMembers(Collections.emptyList())
+                                .setMemberId(UNKNOWN_MEMBER_ID)
+                                .setGenerationId(group.generationId())
+                                .setProtocolName(group.protocolName().orElse(null))
+                                .setProtocolType(group.protocolType().orElse(null))
+                                .setLeader(currentLeader)
+                                .setSkipAssignment(false)
+                                .setErrorCode(Errors.forException(t).code())
+                        );
+                    } else if (context.apiVersion() >= 9) {
+                        // Starting from version 9 of the JoinGroup API, static members are able to
+                        // skip running the assignor based on the `SkipAssignment` field. We leverage
+                        // this to tell the leader that it is the leader of the group but by skipping
+                        // running the assignor while the group is in stable state.
+                        // Notes:
+                        // 1) This allows the leader to continue monitoring metadata changes for the
+                        // group. Note that any metadata changes happening while the static leader is
+                        // down won't be noticed.
+                        // 2) The assignors are not idempotent nor free from side effects. This is why
+                        // we skip entirely the assignment step as it could generate a different group
+                        // assignment which would be ignored by the group coordinator because the group
+                        // is the stable state.
+                        boolean isLeader = group.isLeader(newMemberId);
+
+                        group.completeJoinFuture(newMember, new JoinGroupResponseData()
+                            .setMembers(isLeader ? group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(isLeader)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Prior to version 9 of the JoinGroup API, we wanted to avoid current leader
+                        // performing trivial assignment while the group is in stable stage, because
+                        // the new assignment in leader's next sync call won't be broadcast by a stable group.
+                        // This could be guaranteed by always returning the old leader id so that the current
+                        // leader won't assume itself as a leader based on the returned message, since the new
+                        // member.id won't match returned leader id, therefore no assignment will be performed.
+                        group.completeJoinFuture(newMember, new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(currentLeader)
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+            } else {
+                return maybePrepareRebalanceOrCompleteJoin(
+                    group,
+                    "Group's selectedProtocol will change because static member " +
+                        newMember.memberId() + " with instance id " + groupInstanceId +
+                        " joined with change of protocol; " + "client reason: " + joinReason

Review Comment:
   nit: `" + "` The `+` in this case is not needed.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorResult.java:
##########
@@ -65,6 +100,23 @@ public T response() {
         return response;
     }
 
+    /**
+     * @return The append-future.
+     */
+    public CompletableFuture<T> appendFuture() {
+        return appendFuture;
+    }
+
+    /**
+     * If the append-future exists, this means
+     * that the in-memory state was already updated.
+     *
+     * @return Whether to replay the records.
+     */
+    public boolean shouldReplayRecords() {

Review Comment:
   nit: `replayRecords`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1230,1358 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = groups.get(groupId) == null;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We create records here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()));
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                newGroupResult = new CoordinatorResult<>(records, appendFuture);
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if the group is new, we need to
+                // persist the group to the timeline map.
+                result = newGroupResult;
+            }
+        }
+        return result;
+    }
+
+    private CoordinatorResult<Void, Record> tryCompleteJoinPhase(GenericGroup group) {
+        // Attempt to complete join group phase. We do not complete
+        // the join group phase if this is the initial rebalance.
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(heartbeatKey(groupId, member.memberId()));
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalanceOrCompleteJoin(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalanceOrCompleteJoin(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.previousState() == EMPTY) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalanceOrCompleteJoin(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state. Otherwise, try
+     * to complete the join phase.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybePrepareRebalanceOrCompleteJoin(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        } else {
+            return tryCompleteJoinPhase(group);
+        }
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<Void, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.isInState(EMPTY);
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if (!member.hasAssignment() && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<Void, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        switch (group.currentState()) {
+            case EMPTY:
+            case DEAD:
+                // Always accept the request when the group is empty or dead
+                return true;
+            case PREPARING_REBALANCE:
+                // An existing member is accepted if it is already awaiting. New members are accepted
+                // up to the max group size. Note that the number of awaiting members is used here
+                // for two reasons:
+                // 1) the group size is not reliable as it could already be above the max group size
+                //    if the max group size was reduced.
+                // 2) using the number of awaiting members allows to kick out the last rejoining
+                //    members of the group.
+                return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                    group.numAwaitingJoinResponse() < genericGroupMaxSize;
+            case COMPLETING_REBALANCE:
+            case STABLE:
+                // An existing member is accepted. New members are accepted up to the max group size.
+                // Note that the group size is used here. When the group transitions to CompletingRebalance,
+                // members who haven't rejoined are removed.
+                return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+            default:
+                throw new IllegalStateException("Unknown group state: " + group.stateAsString());
+        }
+    }
+
+    /**
+     * Update a static member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group of the static member.
+     * @param oldMemberId     The existing static member id.
+     * @param newMemberId     The new joining static member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateStaticMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String oldMemberId,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String currentLeader = group.leaderOrNull();
+        GenericGroupMember member = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+
+        // Heartbeat of old member id will expire without effect since the group no longer contains that member id.
+        // New heartbeat shall be scheduled with new member id.
+        rescheduleGenericGroupMemberHeartbeat(group, member);
+
+        int oldRebalanceTimeoutMs = member.rebalanceTimeoutMs();
+        int oldSessionTimeoutMs = member.sessionTimeoutMs();
+        JoinGroupRequestProtocolCollection oldProtocols = member.supportedProtocols();
+
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        if (group.isInState(STABLE)) {
+            // Check if group's selected protocol of next generation will change, if not, simply store group to persist
+            // the updated static member, if yes, rebalance should be triggered to keep the group's assignment
+            // and selected protocol consistent
+            String groupInstanceId = request.groupInstanceId();
+            String selectedProtocolForNextGeneration = group.selectProtocol();
+            if (group.protocolName().orElse("").equals(selectedProtocolForNextGeneration)) {
+                log.info("Static member which joins during Stable stage and doesn't affect " +
+                    "the selected protocol will not trigger a rebalance.");
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    // Starting from version 9 of the JoinGroup API, static members are able to
+                    // skip running the assignor based on the `SkipAssignment` field. We leverage
+                    // this to tell the leader that it is the leader of the group but by skipping
+                    // running the assignor while the group is in stable state.
+                    // Notes:
+                    // 1) This allows the leader to continue monitoring metadata changes for the
+                    // group. Note that any metadata changes happening while the static leader is
+                    // down won't be noticed.
+                    // 2) The assignors are not idempotent nor free from side effects. This is why
+                    // we skip entirely the assignment step as it could generate a different group
+                    // assignment which would be ignored by the group coordinator because the group
+                    // is the stable state.
+                    boolean supportSkippingAssignment = context.apiVersion() >= 9;
+
+                    if (t != null) {
+                        log.warn("Failed to persist metadata for group {}: {}", group.groupId(), t.getMessage());
+
+                        // Failed to persist the member id of the given static member, revert the update of the static member in the group.
+                        group.updateMember(member, oldProtocols, oldRebalanceTimeoutMs, oldSessionTimeoutMs, null);
+                        GenericGroupMember oldMember = group.replaceStaticMember(groupInstanceId, newMemberId, oldMemberId);
+                        rescheduleGenericGroupMemberHeartbeat(group, oldMember);
+
+                        short errorCode = Errors.forException(t).code();
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setMembers(Collections.emptyList())
+                                .setMemberId(UNKNOWN_MEMBER_ID)
+                                .setGenerationId(group.generationId())
+                                .setProtocolName(group.protocolName().orElse(null))
+                                .setProtocolType(group.protocolType().orElse(null))
+                                .setLeader(currentLeader)
+                                .setSkipAssignment(false)
+                                .setErrorCode(errorCode)
+                        );
+
+                    } else if (supportSkippingAssignment) {
+                        boolean isLeader = group.isLeader(newMemberId);
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (isLeader) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        group.completeJoinFuture(member, new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(isLeader)
+                            .setErrorCode(Errors.NONE.code())

Review Comment:
   I would remove them.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1221,1331 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We create records here

Review Comment:
   nit: `data structure`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -286,6 +425,45 @@ ConsumerGroup getOrMaybeCreateConsumerGroup(
         }
     }
 
+    /**
+     * Gets or maybe creates a generic group.
+     *
+     * @param groupId           The group id.
+     * @param createIfNotExists A boolean indicating whether the group should be
+     *                          created if it does not exist.
+     *
+     * @return A GenericGroup.
+     * @throws GroupIdNotFoundException if the group does not exist and createIfNotExists is false or
+     *                                  if the group is not a consumer group.
+     *
+     * Package private for testing.
+     */
+    GenericGroup getOrMaybeCreateGenericGroup(
+        String groupId,
+        boolean createIfNotExists
+    ) {

Review Comment:
   nit: Should we add `throws GroupIdNotFoundException`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1221,1331 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We create records here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                newGroupResult = new CoordinatorResult<>(records, appendFuture);
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if the group is new, we need to
+                // persist the group to the timeline map.
+                result = newGroupResult;
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Attempt to complete join group phase. We do not complete
+     * the join group phase if this is the initial rebalance.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteJoinPhase(GenericGroup group) {

Review Comment:
   nit: `maybe...`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -523,9 +537,17 @@ public void run() {
          */
         @Override
         public void complete(Throwable exception) {
+            boolean appendFutureExists = result != null && result.appendFuture() != null;

Review Comment:
   I was wondering if the following is simpler:
   
   ```
   CompletableFuture appendFuture = result != null ? result.appendFuture() : null;
   ```
   
   then
   
   ```
   if (appendFuture != null) appendFuture.complete(...);`
   
   if (appendFuture != null) appendFuture.completeExceptionally(...);`
   ```
   
   I leave it up to you.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1221,1331 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We create records here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                newGroupResult = new CoordinatorResult<>(records, appendFuture);
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if the group is new, we need to
+                // persist the group to the timeline map.
+                result = newGroupResult;
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Attempt to complete join group phase. We do not complete
+     * the join group phase if this is the initial rebalance.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteJoinPhase(GenericGroup group) {
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberThenRebalanceOrCompleteJoin(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(group.isLeader(memberId) ?
+                                group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}) with {} members",
+                    groupId, group.generationId(), topicPartition, group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(heartbeatKey(groupId, member.memberId()));
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalanceOrCompleteJoin(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member. Then begin a rebalance or complete the join phase.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateMemberThenRebalanceOrCompleteJoin(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalanceOrCompleteJoin(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.previousState() == EMPTY) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalanceOrCompleteJoin(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state. Otherwise, try
+     * to complete the join phase.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybePrepareRebalanceOrCompleteJoin(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        } else {
+            return tryCompleteJoinPhase(group);
+        }
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     *
+     * Package private for testing.
+     */
+    CoordinatorResult<Void, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.isInState(EMPTY);
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : maybeCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if (!member.hasAssignment() && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<Void, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        switch (group.currentState()) {
+            case EMPTY:
+            case DEAD:
+                // Always accept the request when the group is empty or dead
+                return true;
+            case PREPARING_REBALANCE:
+                // An existing member is accepted if it is already awaiting. New members are accepted
+                // up to the max group size. Note that the number of awaiting members is used here
+                // for two reasons:
+                // 1) the group size is not reliable as it could already be above the max group size
+                //    if the max group size was reduced.
+                // 2) using the number of awaiting members allows to kick out the last rejoining
+                //    members of the group.
+                return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                    group.numAwaitingJoinResponse() < genericGroupMaxSize;
+            case COMPLETING_REBALANCE:
+            case STABLE:
+                // An existing member is accepted. New members are accepted up to the max group size.
+                // Note that the group size is used here. When the group transitions to CompletingRebalance,
+                // members who haven't rejoined are removed.
+                return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+            default:
+                throw new IllegalStateException("Unknown group state: " + group.stateAsString());
+        }
+    }
+
+    /**
+     * Update a static member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group of the static member.
+     * @param oldMemberId     The existing static member id.
+     * @param newMemberId     The new joining static member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateStaticMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String oldMemberId,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String currentLeader = group.leaderOrNull();
+        GenericGroupMember newMember = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+
+        // Heartbeat of old member id will expire without effect since the group no longer contains that member id.
+        // New heartbeat shall be scheduled with new member id.
+        rescheduleGenericGroupMemberHeartbeat(group, newMember);
+
+        int oldRebalanceTimeoutMs = newMember.rebalanceTimeoutMs();
+        int oldSessionTimeoutMs = newMember.sessionTimeoutMs();
+        JoinGroupRequestProtocolCollection oldProtocols = newMember.supportedProtocols();
+
+        group.updateMember(
+            newMember,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        if (group.isInState(STABLE)) {
+            // Check if group's selected protocol of next generation will change, if not, simply store group to persist
+            // the updated static member, if yes, rebalance should be triggered to keep the group's assignment
+            // and selected protocol consistent
+            String groupInstanceId = request.groupInstanceId();
+            String selectedProtocolForNextGeneration = group.selectProtocol();
+            if (group.protocolName().orElse("").equals(selectedProtocolForNextGeneration)) {
+                log.info("Static member which joins during Stable stage and doesn't affect " +
+                    "the selected protocol will not trigger a rebalance.");
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        log.warn("Failed to persist metadata for group {}: {}", group.groupId(), t.getMessage());

Review Comment:
   nit: Could we expand this error a little? It would be great if it could capture that it failed to update the metadata for a static member, etc.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1221,1331 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We create records here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                newGroupResult = new CoordinatorResult<>(records, appendFuture);
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if the group is new, we need to
+                // persist the group to the timeline map.
+                result = newGroupResult;
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Attempt to complete join group phase. We do not complete
+     * the join group phase if this is the initial rebalance.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteJoinPhase(GenericGroup group) {
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberThenRebalanceOrCompleteJoin(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(group.isLeader(memberId) ?
+                                group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}) with {} members",
+                    groupId, group.generationId(), topicPartition, group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(heartbeatKey(groupId, member.memberId()));
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalanceOrCompleteJoin(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member. Then begin a rebalance or complete the join phase.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateMemberThenRebalanceOrCompleteJoin(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalanceOrCompleteJoin(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.previousState() == EMPTY) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalanceOrCompleteJoin(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state. Otherwise, try
+     * to complete the join phase.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybePrepareRebalanceOrCompleteJoin(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        } else {
+            return tryCompleteJoinPhase(group);
+        }
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     *
+     * Package private for testing.
+     */
+    CoordinatorResult<Void, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.isInState(EMPTY);
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : maybeCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if (!member.hasAssignment() && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<Void, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        switch (group.currentState()) {
+            case EMPTY:
+            case DEAD:
+                // Always accept the request when the group is empty or dead
+                return true;
+            case PREPARING_REBALANCE:
+                // An existing member is accepted if it is already awaiting. New members are accepted
+                // up to the max group size. Note that the number of awaiting members is used here
+                // for two reasons:
+                // 1) the group size is not reliable as it could already be above the max group size
+                //    if the max group size was reduced.
+                // 2) using the number of awaiting members allows to kick out the last rejoining
+                //    members of the group.
+                return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                    group.numAwaitingJoinResponse() < genericGroupMaxSize;
+            case COMPLETING_REBALANCE:
+            case STABLE:
+                // An existing member is accepted. New members are accepted up to the max group size.
+                // Note that the group size is used here. When the group transitions to CompletingRebalance,
+                // members who haven't rejoined are removed.
+                return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+            default:
+                throw new IllegalStateException("Unknown group state: " + group.stateAsString());
+        }
+    }
+
+    /**
+     * Update a static member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group of the static member.
+     * @param oldMemberId     The existing static member id.
+     * @param newMemberId     The new joining static member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateStaticMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String oldMemberId,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String currentLeader = group.leaderOrNull();
+        GenericGroupMember newMember = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+
+        // Heartbeat of old member id will expire without effect since the group no longer contains that member id.
+        // New heartbeat shall be scheduled with new member id.
+        rescheduleGenericGroupMemberHeartbeat(group, newMember);
+
+        int oldRebalanceTimeoutMs = newMember.rebalanceTimeoutMs();
+        int oldSessionTimeoutMs = newMember.sessionTimeoutMs();
+        JoinGroupRequestProtocolCollection oldProtocols = newMember.supportedProtocols();
+
+        group.updateMember(
+            newMember,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        if (group.isInState(STABLE)) {
+            // Check if group's selected protocol of next generation will change, if not, simply store group to persist
+            // the updated static member, if yes, rebalance should be triggered to keep the group's assignment
+            // and selected protocol consistent
+            String groupInstanceId = request.groupInstanceId();
+            String selectedProtocolForNextGeneration = group.selectProtocol();
+            if (group.protocolName().orElse("").equals(selectedProtocolForNextGeneration)) {
+                log.info("Static member which joins during Stable stage and doesn't affect " +
+                    "the selected protocol will not trigger a rebalance.");
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        log.warn("Failed to persist metadata for group {}: {}", group.groupId(), t.getMessage());
+
+                        // Failed to persist the member id of the given static member, revert the update of the static member in the group.
+                        group.updateMember(newMember, oldProtocols, oldRebalanceTimeoutMs, oldSessionTimeoutMs, null);
+                        GenericGroupMember oldMember = group.replaceStaticMember(groupInstanceId, newMemberId, oldMemberId);
+                        rescheduleGenericGroupMemberHeartbeat(group, oldMember);
+
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setMembers(Collections.emptyList())
+                                .setMemberId(UNKNOWN_MEMBER_ID)
+                                .setGenerationId(group.generationId())
+                                .setProtocolName(group.protocolName().orElse(null))
+                                .setProtocolType(group.protocolType().orElse(null))
+                                .setLeader(currentLeader)
+                                .setSkipAssignment(false)
+                                .setErrorCode(Errors.forException(t).code())
+                        );
+                    } else if (context.apiVersion() >= 9) {
+                        // Starting from version 9 of the JoinGroup API, static members are able to
+                        // skip running the assignor based on the `SkipAssignment` field. We leverage
+                        // this to tell the leader that it is the leader of the group but by skipping
+                        // running the assignor while the group is in stable state.
+                        // Notes:
+                        // 1) This allows the leader to continue monitoring metadata changes for the
+                        // group. Note that any metadata changes happening while the static leader is
+                        // down won't be noticed.
+                        // 2) The assignors are not idempotent nor free from side effects. This is why
+                        // we skip entirely the assignment step as it could generate a different group
+                        // assignment which would be ignored by the group coordinator because the group
+                        // is the stable state.
+                        boolean isLeader = group.isLeader(newMemberId);
+
+                        group.completeJoinFuture(newMember, new JoinGroupResponseData()
+                            .setMembers(isLeader ? group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(isLeader)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Prior to version 9 of the JoinGroup API, we wanted to avoid current leader
+                        // performing trivial assignment while the group is in stable stage, because
+                        // the new assignment in leader's next sync call won't be broadcast by a stable group.
+                        // This could be guaranteed by always returning the old leader id so that the current
+                        // leader won't assume itself as a leader based on the returned message, since the new
+                        // member.id won't match returned leader id, therefore no assignment will be performed.
+                        group.completeJoinFuture(newMember, new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(currentLeader)
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+            } else {
+                return maybePrepareRebalanceOrCompleteJoin(
+                    group,
+                    "Group's selectedProtocol will change because static member " +
+                        newMember.memberId() + " with instance id " + groupInstanceId +
+                        " joined with change of protocol; " + "client reason: " + joinReason
+                );
+            }
+        } else if (group.isInState(COMPLETING_REBALANCE)) {
+            // if the group is in after-sync stage, upon getting a new join-group of a known static member
+            // we should still trigger a new rebalance, since the old member may already be sent to the leader
+            // for assignment, and hence when the assignment gets back there would be a mismatch of the old member id
+            // with the new replaced member id. As a result the new member id would not get any assignment.
+            return prepareRebalance(
+                group,
+                "Updating metadata for static member " + newMember.memberId() + " with instance id " +
+                    request.groupInstanceId() + "; client reason: " + joinReason

Review Comment:
   I just noticed that whenever we use `joinReason`, we also have the `JoinGroupRequestData`. How about adding a helper in `JoinRequest` class to get the reason from `JoinGroupRequestData`? Then, we don't have to pass it anymore to all the methods and we can remove the logic to compute it in `genericGroupJoin`. What do you think?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1230,1358 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = groups.get(groupId) == null;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (isNewGroup) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure. We create records here
+                // while the group is still empty.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()));
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                newGroupResult = new CoordinatorResult<>(records, appendFuture);
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if the group is new, we need to
+                // persist the group to the timeline map.
+                result = newGroupResult;
+            }
+        }
+        return result;
+    }
+
+    private CoordinatorResult<Void, Record> tryCompleteJoinPhase(GenericGroup group) {
+        // Attempt to complete join group phase. We do not complete
+        // the join group phase if this is the initial rebalance.
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(heartbeatKey(groupId, member.memberId()));
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalanceOrCompleteJoin(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalanceOrCompleteJoin(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.previousState() == EMPTY) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalanceOrCompleteJoin(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state. Otherwise, try
+     * to complete the join phase.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybePrepareRebalanceOrCompleteJoin(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        } else {
+            return tryCompleteJoinPhase(group);
+        }
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<Void, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.isInState(EMPTY);
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if (!member.hasAssignment() && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<Void, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        switch (group.currentState()) {
+            case EMPTY:
+            case DEAD:
+                // Always accept the request when the group is empty or dead
+                return true;
+            case PREPARING_REBALANCE:
+                // An existing member is accepted if it is already awaiting. New members are accepted
+                // up to the max group size. Note that the number of awaiting members is used here
+                // for two reasons:
+                // 1) the group size is not reliable as it could already be above the max group size
+                //    if the max group size was reduced.
+                // 2) using the number of awaiting members allows to kick out the last rejoining
+                //    members of the group.
+                return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                    group.numAwaitingJoinResponse() < genericGroupMaxSize;
+            case COMPLETING_REBALANCE:
+            case STABLE:
+                // An existing member is accepted. New members are accepted up to the max group size.
+                // Note that the group size is used here. When the group transitions to CompletingRebalance,
+                // members who haven't rejoined are removed.
+                return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+            default:
+                throw new IllegalStateException("Unknown group state: " + group.stateAsString());

Review Comment:
   We can keep it, I suppose.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java:
##########
@@ -271,4 +289,110 @@ public void testOnResignation() {
             10
         );
     }
+
+    @Test
+    public void testJoinGroup() {
+        CoordinatorRuntime<ReplicatedGroupCoordinator, Record> runtime = mockRuntime();
+        GroupCoordinatorService service = new GroupCoordinatorService(
+            new LogContext(),
+            createConfig(),
+            runtime
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestData()
+            .setGroupId("foo");
+
+        service.startup(() -> 1);
+
+        when(runtime.scheduleWriteOperation(
+            ArgumentMatchers.eq("generic-group-join"),
+            ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
+            ArgumentMatchers.any()
+        )).thenReturn(CompletableFuture.completedFuture(
+            new JoinGroupResponseData()
+        ));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = service.joinGroup(
+            requestContext(ApiKeys.JOIN_GROUP),
+            request,
+            BufferSupplier.NO_CACHING
+        );
+
+        assertFalse(responseFuture.isDone());

Review Comment:
   Why is this false?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java:
##########
@@ -271,4 +289,110 @@ public void testOnResignation() {
             10
         );
     }
+
+    @Test
+    public void testJoinGroup() {
+        CoordinatorRuntime<ReplicatedGroupCoordinator, Record> runtime = mockRuntime();
+        GroupCoordinatorService service = new GroupCoordinatorService(
+            new LogContext(),
+            createConfig(),
+            runtime
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestData()
+            .setGroupId("foo");
+
+        service.startup(() -> 1);
+
+        when(runtime.scheduleWriteOperation(
+            ArgumentMatchers.eq("generic-group-join"),
+            ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
+            ArgumentMatchers.any()
+        )).thenReturn(CompletableFuture.completedFuture(
+            new JoinGroupResponseData()
+        ));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = service.joinGroup(
+            requestContext(ApiKeys.JOIN_GROUP),
+            request,
+            BufferSupplier.NO_CACHING
+        );
+
+        assertFalse(responseFuture.isDone());
+    }
+
+    @Test
+    public void testJoinGroupWithException() throws Exception {
+        CoordinatorRuntime<ReplicatedGroupCoordinator, Record> runtime = mockRuntime();
+        GroupCoordinatorService service = new GroupCoordinatorService(
+            new LogContext(),
+            createConfig(),
+            runtime
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestData()
+            .setGroupId("foo");
+
+        service.startup(() -> 1);
+
+        when(runtime.scheduleWriteOperation(
+            ArgumentMatchers.eq("generic-group-join"),
+            ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
+            ArgumentMatchers.any()
+        )).thenReturn(FutureUtils.failedFuture(new IllegalStateException()));
+
+        CompletableFuture<JoinGroupResponseData> future = service.joinGroup(
+            requestContext(ApiKeys.JOIN_GROUP),
+            request,
+            BufferSupplier.NO_CACHING
+        );
+
+        assertEquals(
+            new JoinGroupResponseData()
+                .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()),
+            future.get(5, TimeUnit.SECONDS)
+        );
+    }
+
+    @Test
+    public void testJoinGroupInvalidGroupId() throws Exception {
+        CoordinatorRuntime<ReplicatedGroupCoordinator, Record> runtime = mockRuntime();
+        GroupCoordinatorService service = new GroupCoordinatorService(
+            new LogContext(),
+            createConfig(),
+            runtime
+        );
+
+        service.startup(() -> 1);
+
+        JoinGroupRequestData request = new JoinGroupRequestData()
+            .setGroupId(null)
+            .setMemberId(UNKNOWN_MEMBER_ID);
+
+        RequestContext context = new RequestContext(
+            new RequestHeader(
+                ApiKeys.JOIN_GROUP,
+                ApiKeys.JOIN_GROUP.latestVersion(),
+                "client",
+                0
+            ),
+            "1",
+            InetAddress.getLoopbackAddress(),
+            KafkaPrincipal.ANONYMOUS,
+            ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
+            SecurityProtocol.PLAINTEXT,
+            ClientInformation.EMPTY,
+            false
+        );
+
+        CompletableFuture<JoinGroupResponseData> response = service.joinGroup(
+            context,
+            request,
+            BufferSupplier.NO_CACHING
+        );
+
+        assertTrue(response.isDone());
+        assertEquals(Errors.INVALID_GROUP_ID.code(), response.get().errorCode());
+        assertEquals(UNKNOWN_MEMBER_ID, response.get().memberId());

Review Comment:
   It may be better to validate the full response here.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -228,14 +261,21 @@ public List<Record> build(TopicsImage topicsImage) {
 
     static class GroupMetadataManagerTestContext {
         static class Builder {
-            final private Time time = new MockTime();

Review Comment:
   Could we revert this?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -367,6 +446,155 @@ public CoordinatorResult<ConsumerGroupHeartbeatResponseData, Record> consumerGro
             return result;
         }
 
+        public CompletableFuture<JoinGroupResponseData> sendGenericGroupJoin(
+            JoinGroupRequestData request
+        ) {
+            return sendGenericGroupJoin(request, false);
+        }
+
+        public CompletableFuture<JoinGroupResponseData> sendGenericGroupJoin(
+            JoinGroupRequestData request,
+            boolean requireKnownMemberId
+        ) {
+            return sendGenericGroupJoin(request, requireKnownMemberId, false, null);
+        }
+
+        public CompletableFuture<JoinGroupResponseData> sendGenericGroupJoin(
+            JoinGroupRequestData request,
+            boolean requireKnownMemberId,
+            boolean supportSkippingAssignment,
+            ExpectedGenericGroupResult expectedResult
+        ) {
+            // requireKnownMemberId is true: version >= 4
+            // supportSkippingAssignment is true: version >= 9
+            short joinGroupVersion = 3;
+
+            if (requireKnownMemberId) {
+                joinGroupVersion = 4;
+                if (supportSkippingAssignment) {
+                    joinGroupVersion = ApiKeys.JOIN_GROUP.latestVersion();
+                }
+            }
+
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+            RequestContext context = new RequestContext(
+                new RequestHeader(
+                    ApiKeys.JOIN_GROUP,
+                    joinGroupVersion,
+                    "client",
+                    0
+                ),
+                "1",
+                InetAddress.getLoopbackAddress(),
+                KafkaPrincipal.ANONYMOUS,
+                ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
+                SecurityProtocol.PLAINTEXT,
+                ClientInformation.EMPTY,
+                false
+            );
+
+            CoordinatorResult<Void, Record> result = groupMetadataManager.genericGroupJoin(
+                context,
+                request,
+                responseFuture
+            );
+
+            if (expectedResult != null) {
+                GenericGroup group = groupMetadataManager.getOrMaybeCreateGenericGroup(
+                    request.groupId(),
+                    false
+                );
+
+                Record groupMetadataRecord;
+                if (expectedResult.isNewGroup) {
+                    groupMetadataRecord = newEmptyGroupMetadataRecord(group, MetadataVersion.latest());
+                } else {
+                    groupMetadataRecord = RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest());
+                }
+
+                expectedResult.records = Collections.singletonList(groupMetadataRecord);

Review Comment:
   This feels weird.... You pass the expected result and you alter it here. I think that it would be better to separate concerns and to return records and future to the caller and to let it do the validation.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {

Review Comment:
   nit: `IntStream.range(0, groupMaxSize + 1)` that you just used above is pretty nice. I wonder if we could use it here as well.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }
-    }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() {

Review Comment:
   In Scala, this test runs with a dynamic member and a static member. We don't do the static part here. Why?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }

Review Comment:
   nit: You should try to use the Stream API more often.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }
-    }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, new ArrayList<>(group.allMembers()).get(0).memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());
+        assertNull(response.protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertFalse(responseFuture.isDone());
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+        context.timer.advanceClock(2);
+
+        // Advance clock past initial rebalance delay and verify futures are not completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+
+        // Advance clock beyond recomputed delay and make sure the futures have completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldDelayRebalanceUptoRebalanceTimeout() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs + 1);
+
+        CompletableFuture<JoinGroupResponseData> thirdMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        // Advance clock right before rebalance timeout.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+        assertFalse(thirdMemberResponseFuture.isDone());
+
+        // Advance clock beyond rebalance timeout.
+        context.timer.advanceClock(1);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertTrue(thirdMemberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupReplaceStaticMember() throws Exception {

Review Comment:
   Is this a new test?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));

Review Comment:
   Why do we need to handle the first request differently? Is it because it may generate a record? If so, I would not do this in every tests but only in one test focused on this.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );

Review Comment:
   Do we need this?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -129,7 +161,8 @@ public MetadataImageBuilder addTopic(
             String topicName,
             int numPartitions
         ) {
-            delta.replay(new TopicRecord().setTopicId(topicId).setName(topicName));
+            delta.getOrCreateTopicsDelta();
+            delta.topicsDelta().replay(new TopicRecord().setTopicId(topicId).setName(topicName));

Review Comment:
   Should we revert this?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -228,14 +261,21 @@ public List<Record> build(TopicsImage topicsImage) {
 
     static class GroupMetadataManagerTestContext {
         static class Builder {
-            final private Time time = new MockTime();
             final private LogContext logContext = new LogContext();
             final private SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
             private MetadataImage metadataImage;
-            private List<PartitionAssignor> assignors;

Review Comment:
   Why are we changing this?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -262,23 +302,53 @@ public Builder withConsumerGroupMetadataRefreshIntervalMs(int consumerGroupMetad
                 return this;
             }
 
+            public Builder withGenericGroupMaxSize(int genericGroupMaxSize) {
+                this.genericGroupMaxSize = genericGroupMaxSize;
+                return this;
+            }
+

Review Comment:
   nit: Empty line.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+

Review Comment:
   nit: Empty line.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );

Review Comment:
   Is this needed? I would have thought that the group should have been created by the first request.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java:
##########
@@ -271,4 +289,110 @@ public void testOnResignation() {
             10
         );
     }
+
+    @Test
+    public void testJoinGroup() {
+        CoordinatorRuntime<ReplicatedGroupCoordinator, Record> runtime = mockRuntime();
+        GroupCoordinatorService service = new GroupCoordinatorService(
+            new LogContext(),
+            createConfig(),
+            runtime
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestData()
+            .setGroupId("foo");
+
+        service.startup(() -> 1);
+
+        when(runtime.scheduleWriteOperation(
+            ArgumentMatchers.eq("generic-group-join"),
+            ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
+            ArgumentMatchers.any()
+        )).thenReturn(CompletableFuture.completedFuture(
+            new JoinGroupResponseData()
+        ));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = service.joinGroup(
+            requestContext(ApiKeys.JOIN_GROUP),
+            request,
+            BufferSupplier.NO_CACHING
+        );
+
+        assertFalse(responseFuture.isDone());
+    }
+
+    @Test
+    public void testJoinGroupWithException() throws Exception {
+        CoordinatorRuntime<ReplicatedGroupCoordinator, Record> runtime = mockRuntime();
+        GroupCoordinatorService service = new GroupCoordinatorService(
+            new LogContext(),
+            createConfig(),
+            runtime
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestData()
+            .setGroupId("foo");
+
+        service.startup(() -> 1);
+
+        when(runtime.scheduleWriteOperation(
+            ArgumentMatchers.eq("generic-group-join"),
+            ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
+            ArgumentMatchers.any()
+        )).thenReturn(FutureUtils.failedFuture(new IllegalStateException()));
+
+        CompletableFuture<JoinGroupResponseData> future = service.joinGroup(
+            requestContext(ApiKeys.JOIN_GROUP),
+            request,
+            BufferSupplier.NO_CACHING
+        );
+
+        assertEquals(
+            new JoinGroupResponseData()
+                .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()),
+            future.get(5, TimeUnit.SECONDS)
+        );
+    }
+
+    @Test
+    public void testJoinGroupInvalidGroupId() throws Exception {
+        CoordinatorRuntime<ReplicatedGroupCoordinator, Record> runtime = mockRuntime();
+        GroupCoordinatorService service = new GroupCoordinatorService(
+            new LogContext(),
+            createConfig(),
+            runtime
+        );
+
+        service.startup(() -> 1);
+
+        JoinGroupRequestData request = new JoinGroupRequestData()
+            .setGroupId(null)
+            .setMemberId(UNKNOWN_MEMBER_ID);
+
+        RequestContext context = new RequestContext(
+            new RequestHeader(
+                ApiKeys.JOIN_GROUP,
+                ApiKeys.JOIN_GROUP.latestVersion(),
+                "client",
+                0
+            ),
+            "1",
+            InetAddress.getLoopbackAddress(),
+            KafkaPrincipal.ANONYMOUS,
+            ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
+            SecurityProtocol.PLAINTEXT,
+            ClientInformation.EMPTY,
+            false
+        );
+
+        CompletableFuture<JoinGroupResponseData> response = service.joinGroup(
+            context,
+            request,
+            BufferSupplier.NO_CACHING
+        );
+
+        assertTrue(response.isDone());
+        assertEquals(Errors.INVALID_GROUP_ID.code(), response.get().errorCode());
+        assertEquals(UNKNOWN_MEMBER_ID, response.get().memberId());
+    }
 }

Review Comment:
   Do we need to add tests for the errors convertion?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );

Review Comment:
   Is this needed as well?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -367,6 +446,155 @@ public CoordinatorResult<ConsumerGroupHeartbeatResponseData, Record> consumerGro
             return result;
         }
 
+        public CompletableFuture<JoinGroupResponseData> sendGenericGroupJoin(
+            JoinGroupRequestData request
+        ) {
+            return sendGenericGroupJoin(request, false);
+        }
+
+        public CompletableFuture<JoinGroupResponseData> sendGenericGroupJoin(
+            JoinGroupRequestData request,
+            boolean requireKnownMemberId
+        ) {
+            return sendGenericGroupJoin(request, requireKnownMemberId, false, null);
+        }
+
+        public CompletableFuture<JoinGroupResponseData> sendGenericGroupJoin(
+            JoinGroupRequestData request,
+            boolean requireKnownMemberId,
+            boolean supportSkippingAssignment,
+            ExpectedGenericGroupResult expectedResult
+        ) {
+            // requireKnownMemberId is true: version >= 4
+            // supportSkippingAssignment is true: version >= 9
+            short joinGroupVersion = 3;
+
+            if (requireKnownMemberId) {
+                joinGroupVersion = 4;
+                if (supportSkippingAssignment) {
+                    joinGroupVersion = ApiKeys.JOIN_GROUP.latestVersion();
+                }
+            }
+
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+            RequestContext context = new RequestContext(
+                new RequestHeader(
+                    ApiKeys.JOIN_GROUP,
+                    joinGroupVersion,
+                    "client",
+                    0
+                ),
+                "1",
+                InetAddress.getLoopbackAddress(),
+                KafkaPrincipal.ANONYMOUS,
+                ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
+                SecurityProtocol.PLAINTEXT,
+                ClientInformation.EMPTY,
+                false
+            );
+
+            CoordinatorResult<Void, Record> result = groupMetadataManager.genericGroupJoin(
+                context,
+                request,
+                responseFuture
+            );
+
+            if (expectedResult != null) {
+                GenericGroup group = groupMetadataManager.getOrMaybeCreateGenericGroup(
+                    request.groupId(),
+                    false
+                );
+
+                Record groupMetadataRecord;
+                if (expectedResult.isNewGroup) {
+                    groupMetadataRecord = newEmptyGroupMetadataRecord(group, MetadataVersion.latest());
+                } else {
+                    groupMetadataRecord = RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest());
+                }
+
+                expectedResult.records = Collections.singletonList(groupMetadataRecord);
+            }
+
+            verifyCoordinatorResult(expectedResult, result);
+            return responseFuture;
+        }
+
+        public JoinGroupResponseData joinGenericGroupAsDynamicMember(
+            JoinGroupRequestData request
+        ) {
+            boolean requireKnownMemberId = true;
+            String newMemberId = request.memberId();
+
+            try {
+                if (request.memberId().equals(UNKNOWN_MEMBER_ID)) {
+                    // Since member id is required, we need another round to get the successful join group result.
+                    CompletableFuture<JoinGroupResponseData> responseFuture = sendGenericGroupJoin(
+                        request,
+                        requireKnownMemberId,
+                        false,
+                        new ExpectedGenericGroupResult(Errors.NONE, true)
+                    );
+                    assertTrue(responseFuture.isDone());
+                    short errorCode = responseFuture.get(5, TimeUnit.SECONDS).errorCode();
+                    if (errorCode != Errors.MEMBER_ID_REQUIRED.code()) {
+                        // If some other error is triggered, return the error immediately for caller to handle.
+                        return responseFuture.get(5, TimeUnit.SECONDS);
+                    }
+                    newMemberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+                }
+
+                // Second round
+                JoinGroupRequestData secondRequest = new JoinGroupRequestData()
+                    .setGroupId(request.groupId())
+                    .setMemberId(newMemberId)
+                    .setProtocolType(request.protocolType())
+                    .setProtocols(request.protocols())
+                    .setSessionTimeoutMs(request.sessionTimeoutMs())
+                    .setRebalanceTimeoutMs(request.rebalanceTimeoutMs())
+                    .setReason(request.reason());
+
+                CompletableFuture<JoinGroupResponseData> responseFuture = sendGenericGroupJoin(
+                    secondRequest,
+                    requireKnownMemberId
+                );
+
+                timer.advanceClock(genericGroupInitialRebalanceDelayMs);
+                assertTrue(responseFuture.isDone());
+                return responseFuture.get(5, TimeUnit.SECONDS);
+            } catch (Exception e) {
+                e.printStackTrace();
+            }

Review Comment:
   Do we need this? If we do, we should replace `e.printStackTrace();`.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -367,6 +446,155 @@ public CoordinatorResult<ConsumerGroupHeartbeatResponseData, Record> consumerGro
             return result;
         }
 
+        public CompletableFuture<JoinGroupResponseData> sendGenericGroupJoin(
+            JoinGroupRequestData request
+        ) {
+            return sendGenericGroupJoin(request, false);
+        }
+
+        public CompletableFuture<JoinGroupResponseData> sendGenericGroupJoin(
+            JoinGroupRequestData request,
+            boolean requireKnownMemberId
+        ) {
+            return sendGenericGroupJoin(request, requireKnownMemberId, false, null);
+        }
+
+        public CompletableFuture<JoinGroupResponseData> sendGenericGroupJoin(
+            JoinGroupRequestData request,
+            boolean requireKnownMemberId,
+            boolean supportSkippingAssignment,
+            ExpectedGenericGroupResult expectedResult
+        ) {
+            // requireKnownMemberId is true: version >= 4
+            // supportSkippingAssignment is true: version >= 9
+            short joinGroupVersion = 3;
+
+            if (requireKnownMemberId) {
+                joinGroupVersion = 4;
+                if (supportSkippingAssignment) {
+                    joinGroupVersion = ApiKeys.JOIN_GROUP.latestVersion();
+                }
+            }
+
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+            RequestContext context = new RequestContext(
+                new RequestHeader(
+                    ApiKeys.JOIN_GROUP,
+                    joinGroupVersion,
+                    "client",
+                    0
+                ),
+                "1",
+                InetAddress.getLoopbackAddress(),
+                KafkaPrincipal.ANONYMOUS,
+                ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
+                SecurityProtocol.PLAINTEXT,
+                ClientInformation.EMPTY,
+                false
+            );
+
+            CoordinatorResult<Void, Record> result = groupMetadataManager.genericGroupJoin(
+                context,
+                request,
+                responseFuture
+            );
+
+            if (expectedResult != null) {
+                GenericGroup group = groupMetadataManager.getOrMaybeCreateGenericGroup(
+                    request.groupId(),
+                    false
+                );
+
+                Record groupMetadataRecord;
+                if (expectedResult.isNewGroup) {
+                    groupMetadataRecord = newEmptyGroupMetadataRecord(group, MetadataVersion.latest());
+                } else {
+                    groupMetadataRecord = RecordHelpers.newGroupMetadataRecord(group, MetadataVersion.latest());
+                }
+
+                expectedResult.records = Collections.singletonList(groupMetadataRecord);
+            }
+
+            verifyCoordinatorResult(expectedResult, result);
+            return responseFuture;
+        }
+
+        public JoinGroupResponseData joinGenericGroupAsDynamicMember(
+            JoinGroupRequestData request
+        ) {
+            boolean requireKnownMemberId = true;
+            String newMemberId = request.memberId();
+
+            try {
+                if (request.memberId().equals(UNKNOWN_MEMBER_ID)) {
+                    // Since member id is required, we need another round to get the successful join group result.
+                    CompletableFuture<JoinGroupResponseData> responseFuture = sendGenericGroupJoin(
+                        request,
+                        requireKnownMemberId,
+                        false,
+                        new ExpectedGenericGroupResult(Errors.NONE, true)
+                    );
+                    assertTrue(responseFuture.isDone());
+                    short errorCode = responseFuture.get(5, TimeUnit.SECONDS).errorCode();
+                    if (errorCode != Errors.MEMBER_ID_REQUIRED.code()) {
+                        // If some other error is triggered, return the error immediately for caller to handle.
+                        return responseFuture.get(5, TimeUnit.SECONDS);
+                    }
+                    newMemberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+                }
+
+                // Second round
+                JoinGroupRequestData secondRequest = new JoinGroupRequestData()
+                    .setGroupId(request.groupId())
+                    .setMemberId(newMemberId)
+                    .setProtocolType(request.protocolType())
+                    .setProtocols(request.protocols())
+                    .setSessionTimeoutMs(request.sessionTimeoutMs())
+                    .setRebalanceTimeoutMs(request.rebalanceTimeoutMs())
+                    .setReason(request.reason());
+
+                CompletableFuture<JoinGroupResponseData> responseFuture = sendGenericGroupJoin(
+                    secondRequest,
+                    requireKnownMemberId
+                );
+
+                timer.advanceClock(genericGroupInitialRebalanceDelayMs);
+                assertTrue(responseFuture.isDone());
+                return responseFuture.get(5, TimeUnit.SECONDS);
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+
+            return null;
+        }
+
+        public JoinGroupResponseData joinGenericGroup(
+            JoinGroupRequestData request,
+            boolean requireKnownMemberId,
+            boolean supportSkippingAssignment
+        ) {
+            if (requireKnownMemberId && request.groupInstanceId().isEmpty()) {
+                return joinGenericGroupAsDynamicMember(request);
+            }
+
+            try {
+                CompletableFuture<JoinGroupResponseData> responseFuture = sendGenericGroupJoin(
+                    request,
+                    requireKnownMemberId,
+                    supportSkippingAssignment,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+
+                timer.advanceClock(genericGroupInitialRebalanceDelayMs);
+                assertTrue(responseFuture.isDone());
+                return responseFuture.get(5, TimeUnit.SECONDS);
+            } catch (Exception e) {
+                e.printStackTrace();
+            }

Review Comment:
   ditto.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );

Review Comment:
   ditto.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }
-    }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, new ArrayList<>(group.allMembers()).get(0).memberId());

Review Comment:
   nit: You can do: `group.allMembers().iterator().next()`.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }
-    }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, new ArrayList<>(group.allMembers()).get(0).memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);

Review Comment:
   There is a + 1 in scala.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }
-    }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());

Review Comment:
   In Scala, we had `UNKNOWN_MEMBER_ID` here. Is the change expected?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }
-    }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);

Review Comment:
   There is a `+ 1` in scala. Don't we need it here?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }
-    }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());

Review Comment:
   We were asserting the leader here.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }
-    }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, new ArrayList<>(group.allMembers()).get(0).memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());

Review Comment:
   There is `UNKNOWN_MEMBER_ID` in scala?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }
-    }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, new ArrayList<>(group.allMembers()).get(0).memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);

Review Comment:
   have replied to a thread above



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2402,143 +2610,2090 @@ public void testOnNewMetadataImage() {
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = genericGroup(context, "group-id");
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = genericGroup(context, "group-id");
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
         }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+
+        // Send as static member join.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setGroupInstanceId("group-instance-id"), true, true, null);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(firstResponse.leader(), firstMemberId);
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, group.allMembers().iterator().next().memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+        assertNull(response.protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertFalse(responseFuture.isDone());
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+        context.timer.advanceClock(2);
+
+        // Advance clock past initial rebalance delay and verify futures are not completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+
+        // Advance clock beyond recomputed delay and make sure the futures have completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldDelayRebalanceUptoRebalanceTimeout() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs + 1);
+
+        CompletableFuture<JoinGroupResponseData> thirdMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        // Advance clock right before rebalance timeout.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+        assertFalse(thirdMemberResponseFuture.isDone());
+
+        // Advance clock beyond rebalance timeout.
+        context.timer.advanceClock(1);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertTrue(thirdMemberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), thirdMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupReplaceStaticMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .build();
+
+
+        // Send join group as static member.
+        CompletableFuture<JoinGroupResponseData> oldMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(oldMemberResponseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Replace static member with new member id. Old member id should be fenced.
+        CompletableFuture<JoinGroupResponseData> newMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        assertFalse(newMemberResponseFuture.isDone());
+        assertTrue(oldMemberResponseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Complete join for new member.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(newMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), newMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemovePendingMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(0, group.size());
+        assertEquals(1, group.numPendingJoinMembers());
+
+        // Advance clock by session timeout. Pending member should be removed from group as heartbeat expires.
+        context.timer.advanceClock(1000);
+        assertEquals(0, group.numPendingJoinMembers());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemoveMember() throws Exception {
+        // Set initial rebalance delay to simulate a long running rebalance.
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupInitialRebalanceDelayMs(10 * 60 * 1000)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        String memberId = group.leaderOrNull();
+        assertEquals(1, group.size());
+
+        // Advance clock by new member join timeout. Member should be removed from group as heartbeat expires.
+        // A group that transitions to Empty after completing join phase will generate records.
+        context.timer.expectResult(heartbeatKey("group-id", memberId), new ExpectedGenericGroupResult(
+            Collections.singletonList(newGroupMetadataRecord("group-id",
+                new GroupMetadataValue()
+                    .setMembers(Collections.emptyList())
+                    .setGeneration(1)
+                    .setLeader(null)
+                    .setProtocolType("consumer")
+                    .setProtocol(null)
+                    .setCurrentStateTimestamp(context.time.milliseconds()),
+                MetadataVersion.latest())),
+            Errors.NONE,
+            false
+        ));
+
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.size());
+    }
+
+    @Test
+    public void testExistingMemberJoinDeadGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertTrue(group.hasMemberId(memberId));
+
+        group.transitionTo(DEAD);
+
+        response = context.joinGenericGroupAsDynamicMember(request.setMemberId(memberId));
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupExistingPendingMemberWithGroupInstanceIdThrowsException() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertTrue(responseFuture.isDone());
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        assertThrows(IllegalStateException.class,
+            () -> context.sendGenericGroupJoin(request.setMemberId(memberId).setGroupInstanceId("group-instance-id"))
+        );
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        GenericGroupMember member = group.member(memberId);
+
+        assertEquals(protocols, member.supportedProtocols());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(1, group.generationId());
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        // Send updated member metadata. This should trigger a rebalance and complete the join phase.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(memberId)
+            .setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+        assertEquals(protocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingLeaderTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertTrue(group.isLeader(memberId));
+        assertEquals(1, group.generationId());
+
+        group.transitionTo(STABLE);
+        // Sending join group as leader should trigger a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with updated metadata. This should trigger a rebalance.
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        JoinGroupRequestData memberRequest = request.setMemberId(memberId).setProtocols(protocols);
+        memberResponseFuture = context.sendGenericGroupJoin(memberRequest);
+
+        assertFalse(memberResponseFuture.isDone());
+
+        // Leader rejoins. This completes the join group phase.
+        leaderResponseFuture = context.sendGenericGroupJoin(request.setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(3, group.generationId());
+        assertEquals(2, group.size());
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberDoesNotTriggerRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with no metadata changes. This does not trigger a rebalance.
+        memberResponseFuture = context.sendGenericGroupJoin(request.setMemberId(memberId));
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, memberResponseFuture.get(5, TimeUnit.SECONDS).generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberInEmptyState() throws Exception {
+        // Existing member joins a group that is in Empty/Dead state. Ask member to rejoin with generation id reset.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(-1, responseFuture.get(5, TimeUnit.SECONDS).generationId());
+    }
+
+    @Test
+    public void testCompleteJoinRemoveNotYetRejoinedDynamicMembers() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.generationId());
+
+        // Add new member. This triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout. This will expire the leader as it has not rejoined.
+        context.timer.advanceClock(1000);
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertTrue(group.hasMemberId(memberResponseFuture.get(5, TimeUnit.SECONDS).memberId()));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseInEmptyStateSkipsRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+        assertEquals(0, group.generationId());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        group.transitionTo(DEAD);
+
+        // Advance clock by initial rebalance delay to complete join phase.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertEquals(0, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseNoMembersRejoinedExtendsJoinPhase() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("first-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(30000)
+            .withRebalanceTimeoutMs(10000)
+            .build();
+
+        // First member joins group and completes join phase.
+        JoinGroupResponseData firstMemberResponse = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), firstMemberResponse.errorCode());
+        String firstMemberId = firstMemberResponse.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        // Second member joins and group goes into rebalancing state.
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setGroupInstanceId("second-instance-id"));
+
+        // First static member rejoins and completes join phase.
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        String secondMemberId = secondMemberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        // Trigger a rebalance. No members rejoined.
+        context.groupMetadataManager.prepareRebalance(group, "trigger rebalance");
+
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(0, group.numAwaitingJoinResponse());
+
+        // Advance clock by rebalance timeout to complete join phase. As long as both members have not
+        // rejoined, we extend the join phase.
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        // Let first and second member rejoin. This should complete the join phase.
+        firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(secondMemberId)
+            .setGroupInstanceId("second-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(3, group.generationId());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testReplaceStaticMemberInStableStateNoError(
+        boolean supportSkippingAssignment
+    ) throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, supportSkippingAssignment);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String oldMemberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. This should update the log with the generated member id.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request
+                .setProtocols(protocols)
+                .setRebalanceTimeoutMs(7000)
+                .setSessionTimeoutMs(4500),
+            true,
+            supportSkippingAssignment,
+            new ExpectedGenericGroupResult(Errors.NONE, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        String newMemberId = group.staticMemberId("group-instance-id");
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMemberId)
+            .setMemberId(newMemberId)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(supportSkippingAssignment)
+            .setErrorCode(Errors.NONE.code());
+
+        if (supportSkippingAssignment) {
+            expectedResponse
+                .setMembers(Collections.singletonList(
+                    new JoinGroupResponseData.JoinGroupResponseMember()
+                        .setMemberId(newMemberId)
+                        .setGroupInstanceId("group-instance-id")
+                        .setMetadata(protocols.find("range").metadata())
+                    ))
+                .setLeader(newMemberId);
+        }
+
+        GenericGroupMember updatedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        assertEquals(newMemberId, updatedMember.memberId());
+        assertEquals(Optional.of("group-instance-id"), updatedMember.groupInstanceId());
+        assertEquals(7000, updatedMember.rebalanceTimeoutMs());
+        assertEquals(4500, updatedMember.sessionTimeoutMs());
+        assertEquals(protocols, updatedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateWithUpdatedProtocolTriggersRebalance() throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. The selected protocol changes and triggers a rebalance.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array())
+        );
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertEquals(2, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateErrors() throws Exception {
+        // If the append future fails, we need to revert the soft state to the original member.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, false);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        GenericGroupMember oldMember = group.member(response.memberId());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID but the append fails. This reverts the soft state of the group.
+        protocols.add(new JoinGroupRequestProtocol()
+                .setName("roundrobin")
+                .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                    Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setSessionTimeoutMs(6000)
+                .setRebalanceTimeoutMs(7000)
+                .setProtocols(protocols),
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMember.memberId())
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code());
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        GenericGroupMember revertedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(oldMember.memberId(), revertedMember.memberId());
+        assertEquals(oldMember.groupInstanceId(), revertedMember.groupInstanceId());
+        assertEquals(oldMember.rebalanceTimeoutMs(), revertedMember.rebalanceTimeoutMs());
+        assertEquals(oldMember.sessionTimeoutMs(), revertedMember.sessionTimeoutMs());
+        assertEquals(oldMember.supportedProtocols(), revertedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInCompletingRebalanceStateTriggersRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID and triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertEquals(2, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+    }
+
+    private <T> void assertUnorderedListEquals(
+        List<T> expected,
+        List<T> actual
+    ) {
+        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
+    }
+
+    private void assertResponseEquals(
+        ConsumerGroupHeartbeatResponseData expected,
+        ConsumerGroupHeartbeatResponseData actual
+    ) {
+        if (!responseEquals(expected, actual)) {
+            assertionFailure()
+                .expected(expected)
+                .actual(actual)
+                .buildAndThrow();
+        }
+    }
+
+    private boolean responseEquals(
+        ConsumerGroupHeartbeatResponseData expected,
+        ConsumerGroupHeartbeatResponseData actual
+    ) {
+        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
+        if (expected.errorCode() != actual.errorCode()) return false;
+        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
+        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
+        if (expected.memberEpoch() != actual.memberEpoch()) return false;
+        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
+        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
+        // Unordered comparison of the assignments.
+        return responseAssignmentEquals(expected.assignment(), actual.assignment());
+    }
+
+    private boolean responseAssignmentEquals(
+        ConsumerGroupHeartbeatResponseData.Assignment expected,
+        ConsumerGroupHeartbeatResponseData.Assignment actual
+    ) {
+        if (expected == actual) return true;
+        if (expected == null) return false;
+        if (actual == null) return false;
+
+        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
+            return false;
+
+        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+    }
+
+    private Map<Uuid, Set<Integer>> fromAssignment(
+        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
+    ) {
+        if (assignment == null) return null;
+
+        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
+        assignment.forEach(topicPartitions -> {
+            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+        });
+        return assignmentMap;
+    }
+
+    private void assertRecordsEquals(
+        List<Record> expectedRecords,
+        List<Record> actualRecords
+    ) {
+        try {
+            assertEquals(expectedRecords.size(), actualRecords.size());
+
+            for (int i = 0; i < expectedRecords.size(); i++) {
+                Record expectedRecord = expectedRecords.get(i);
+                Record actualRecord = actualRecords.get(i);
+                assertRecordEquals(expectedRecord, actualRecord);
+            }
+        } catch (AssertionFailedError e) {
+            assertionFailure()
+                .expected(expectedRecords)
+                .actual(actualRecords)
+                .buildAndThrow();
+        }
+    }
+
+    private void assertRecordEquals(
+        Record expected,
+        Record actual
+    ) {
+        try {
+            assertApiMessageAndVersionEquals(expected.key(), actual.key());
+            assertApiMessageAndVersionEquals(expected.value(), actual.value());
+        } catch (AssertionFailedError e) {
+            assertionFailure()
+                .expected(expected)
+                .actual(actual)
+                .buildAndThrow();
+        }
+    }
+
+    private void assertApiMessageAndVersionEquals(
+        ApiMessageAndVersion expected,
+        ApiMessageAndVersion actual
+    ) {
+        if (expected == actual) return;
+
+        assertEquals(expected.version(), actual.version());
+
+        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
+            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
+            // always guaranteed. Therefore, we need a special comparator.
+            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
+                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
+            ConsumerGroupCurrentMemberAssignmentValue actualValue =
+                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+
+            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
+            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
+            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
+            assertEquals(expectedValue.error(), actualValue.error());
+            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
+            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+
+            // We transform those to Maps before comparing them.
+            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
+                fromTopicPartitions(actualValue.assignedPartitions()));
+            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
+                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
+            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
+                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
+        } else {
+            assertEquals(expected.message(), actual.message());
+        }
+    }
+
+    private Map<Uuid, Set<Integer>> fromTopicPartitions(
+        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
+    ) {
+        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
+        assignment.forEach(topicPartitions -> {
+            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
         });
         return assignmentMap;
     }
+
+    private static GenericGroup genericGroup(GroupMetadataManagerTestContext context, String groupId) {
+        return context.groupMetadataManager.getOrMaybeCreateGenericGroup(groupId, false);
+    }
+
+    private List<String> verifyGenericGroupJoinResponses(
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures,
+        int expectedSuccessCount,
+        Errors expectedFailure
+    ) {
+        int successCount = 0;
+        List<String> memberIds = new ArrayList<>();
+        for (CompletableFuture<JoinGroupResponseData> responseFuture : responseFutures) {
+            if (!responseFuture.isDone()) {
+                fail("All responseFutures should be completed.");
+            }
+            try {
+                if (responseFuture.get(5, TimeUnit.SECONDS).errorCode() == Errors.NONE.code()) {
+                    successCount++;
+                } else {
+                    assertEquals(
+                        expectedFailure.code(),
+                        responseFuture.get(5, TimeUnit.SECONDS).errorCode()
+                    );
+                }
+                memberIds.add(responseFuture.get(5, TimeUnit.SECONDS).memberId());
+            } catch (Exception e) {
+                fail("Unexpected exception: " + e.getMessage());
+            }
+        }
+
+        assertEquals(expectedSuccessCount, successCount);
+        return memberIds;
+    }
+
+    /**
+     * Verify the records that should be appended and complete the append future based on a
+     * configured error. Run any assertions to verify the result of the future completion.
+     *
+     * @param expectedResult  The expected result to compare against.
+     * @param result          The result from expiring a join/heartbeat/sync operation.
+     */
+    private static void verifyCoordinatorResult(
+        ExpectedGenericGroupResult expectedResult,
+        CoordinatorResult<Void, Record> result
+    ) {
+        if (expectedResult == null) {
+            assertEquals(EMPTY_RESULT, result);
+        } else {
+            assertEquals(expectedResult.records, result.records());
+            if (expectedResult.mockError == Errors.NONE) {
+                result.appendFuture().complete(null);
+            } else {
+                result.appendFuture().completeExceptionally(expectedResult.mockError.exception());
+            }
+        }
+    }
+
+    private static class MockCoordinatorTimer

Review Comment:
   yeah, i noticed. i don't mind using the other implementation. looks like java's priority queue does arbitrary ordering for the same priority so they should have the same behavior



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2402,143 +2610,2090 @@ public void testOnNewMetadataImage() {
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = genericGroup(context, "group-id");
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = genericGroup(context, "group-id");
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
         }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+
+        // Send as static member join.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setGroupInstanceId("group-instance-id"), true, true, null);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(firstResponse.leader(), firstMemberId);
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, group.allMembers().iterator().next().memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+        assertNull(response.protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertFalse(responseFuture.isDone());
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+        context.timer.advanceClock(2);
+
+        // Advance clock past initial rebalance delay and verify futures are not completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+
+        // Advance clock beyond recomputed delay and make sure the futures have completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldDelayRebalanceUptoRebalanceTimeout() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs + 1);
+
+        CompletableFuture<JoinGroupResponseData> thirdMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        // Advance clock right before rebalance timeout.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+        assertFalse(thirdMemberResponseFuture.isDone());
+
+        // Advance clock beyond rebalance timeout.
+        context.timer.advanceClock(1);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertTrue(thirdMemberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), thirdMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupReplaceStaticMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .build();
+
+
+        // Send join group as static member.
+        CompletableFuture<JoinGroupResponseData> oldMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(oldMemberResponseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Replace static member with new member id. Old member id should be fenced.
+        CompletableFuture<JoinGroupResponseData> newMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        assertFalse(newMemberResponseFuture.isDone());
+        assertTrue(oldMemberResponseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Complete join for new member.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(newMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), newMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemovePendingMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(0, group.size());
+        assertEquals(1, group.numPendingJoinMembers());
+
+        // Advance clock by session timeout. Pending member should be removed from group as heartbeat expires.
+        context.timer.advanceClock(1000);
+        assertEquals(0, group.numPendingJoinMembers());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemoveMember() throws Exception {
+        // Set initial rebalance delay to simulate a long running rebalance.
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupInitialRebalanceDelayMs(10 * 60 * 1000)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        String memberId = group.leaderOrNull();
+        assertEquals(1, group.size());
+
+        // Advance clock by new member join timeout. Member should be removed from group as heartbeat expires.
+        // A group that transitions to Empty after completing join phase will generate records.
+        context.timer.expectResult(heartbeatKey("group-id", memberId), new ExpectedGenericGroupResult(
+            Collections.singletonList(newGroupMetadataRecord("group-id",
+                new GroupMetadataValue()
+                    .setMembers(Collections.emptyList())
+                    .setGeneration(1)
+                    .setLeader(null)
+                    .setProtocolType("consumer")
+                    .setProtocol(null)
+                    .setCurrentStateTimestamp(context.time.milliseconds()),
+                MetadataVersion.latest())),
+            Errors.NONE,
+            false
+        ));
+
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.size());
+    }
+
+    @Test
+    public void testExistingMemberJoinDeadGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertTrue(group.hasMemberId(memberId));
+
+        group.transitionTo(DEAD);
+
+        response = context.joinGenericGroupAsDynamicMember(request.setMemberId(memberId));
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupExistingPendingMemberWithGroupInstanceIdThrowsException() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertTrue(responseFuture.isDone());
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        assertThrows(IllegalStateException.class,
+            () -> context.sendGenericGroupJoin(request.setMemberId(memberId).setGroupInstanceId("group-instance-id"))
+        );
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        GenericGroupMember member = group.member(memberId);
+
+        assertEquals(protocols, member.supportedProtocols());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(1, group.generationId());
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        // Send updated member metadata. This should trigger a rebalance and complete the join phase.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(memberId)
+            .setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+        assertEquals(protocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingLeaderTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertTrue(group.isLeader(memberId));
+        assertEquals(1, group.generationId());
+
+        group.transitionTo(STABLE);
+        // Sending join group as leader should trigger a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with updated metadata. This should trigger a rebalance.
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        JoinGroupRequestData memberRequest = request.setMemberId(memberId).setProtocols(protocols);
+        memberResponseFuture = context.sendGenericGroupJoin(memberRequest);
+
+        assertFalse(memberResponseFuture.isDone());
+
+        // Leader rejoins. This completes the join group phase.
+        leaderResponseFuture = context.sendGenericGroupJoin(request.setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(3, group.generationId());
+        assertEquals(2, group.size());
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberDoesNotTriggerRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with no metadata changes. This does not trigger a rebalance.
+        memberResponseFuture = context.sendGenericGroupJoin(request.setMemberId(memberId));
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, memberResponseFuture.get(5, TimeUnit.SECONDS).generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberInEmptyState() throws Exception {
+        // Existing member joins a group that is in Empty/Dead state. Ask member to rejoin with generation id reset.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(-1, responseFuture.get(5, TimeUnit.SECONDS).generationId());
+    }
+
+    @Test
+    public void testCompleteJoinRemoveNotYetRejoinedDynamicMembers() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.generationId());
+
+        // Add new member. This triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout. This will expire the leader as it has not rejoined.
+        context.timer.advanceClock(1000);
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertTrue(group.hasMemberId(memberResponseFuture.get(5, TimeUnit.SECONDS).memberId()));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseInEmptyStateSkipsRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+        assertEquals(0, group.generationId());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        group.transitionTo(DEAD);
+
+        // Advance clock by initial rebalance delay to complete join phase.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertEquals(0, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseNoMembersRejoinedExtendsJoinPhase() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("first-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(30000)
+            .withRebalanceTimeoutMs(10000)
+            .build();
+
+        // First member joins group and completes join phase.
+        JoinGroupResponseData firstMemberResponse = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), firstMemberResponse.errorCode());
+        String firstMemberId = firstMemberResponse.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        // Second member joins and group goes into rebalancing state.
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setGroupInstanceId("second-instance-id"));
+
+        // First static member rejoins and completes join phase.
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        String secondMemberId = secondMemberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        // Trigger a rebalance. No members rejoined.
+        context.groupMetadataManager.prepareRebalance(group, "trigger rebalance");
+
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(0, group.numAwaitingJoinResponse());
+
+        // Advance clock by rebalance timeout to complete join phase. As long as both members have not
+        // rejoined, we extend the join phase.
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        // Let first and second member rejoin. This should complete the join phase.
+        firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(secondMemberId)
+            .setGroupInstanceId("second-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(3, group.generationId());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testReplaceStaticMemberInStableStateNoError(
+        boolean supportSkippingAssignment
+    ) throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, supportSkippingAssignment);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String oldMemberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. This should update the log with the generated member id.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request
+                .setProtocols(protocols)
+                .setRebalanceTimeoutMs(7000)
+                .setSessionTimeoutMs(4500),
+            true,
+            supportSkippingAssignment,
+            new ExpectedGenericGroupResult(Errors.NONE, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        String newMemberId = group.staticMemberId("group-instance-id");
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMemberId)
+            .setMemberId(newMemberId)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(supportSkippingAssignment)
+            .setErrorCode(Errors.NONE.code());
+
+        if (supportSkippingAssignment) {
+            expectedResponse
+                .setMembers(Collections.singletonList(
+                    new JoinGroupResponseData.JoinGroupResponseMember()
+                        .setMemberId(newMemberId)
+                        .setGroupInstanceId("group-instance-id")
+                        .setMetadata(protocols.find("range").metadata())
+                    ))
+                .setLeader(newMemberId);
+        }
+
+        GenericGroupMember updatedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        assertEquals(newMemberId, updatedMember.memberId());
+        assertEquals(Optional.of("group-instance-id"), updatedMember.groupInstanceId());
+        assertEquals(7000, updatedMember.rebalanceTimeoutMs());
+        assertEquals(4500, updatedMember.sessionTimeoutMs());
+        assertEquals(protocols, updatedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateWithUpdatedProtocolTriggersRebalance() throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));

Review Comment:
   `sendGenericGroup...` methods send the request and return the future.
   `joinGenericGroup...` methods invoke `sendGenericGroup` methods then advance the timer to move the group to completing rebalance state.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2402,143 +2610,2090 @@ public void testOnNewMetadataImage() {
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = genericGroup(context, "group-id");
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = genericGroup(context, "group-id");
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
         }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+
+        // Send as static member join.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setGroupInstanceId("group-instance-id"), true, true, null);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(firstResponse.leader(), firstMemberId);
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, group.allMembers().iterator().next().memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+        assertNull(response.protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertFalse(responseFuture.isDone());
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+        context.timer.advanceClock(2);
+
+        // Advance clock past initial rebalance delay and verify futures are not completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+
+        // Advance clock beyond recomputed delay and make sure the futures have completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldDelayRebalanceUptoRebalanceTimeout() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs + 1);
+
+        CompletableFuture<JoinGroupResponseData> thirdMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        // Advance clock right before rebalance timeout.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+        assertFalse(thirdMemberResponseFuture.isDone());
+
+        // Advance clock beyond rebalance timeout.
+        context.timer.advanceClock(1);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertTrue(thirdMemberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), thirdMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupReplaceStaticMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .build();
+
+
+        // Send join group as static member.
+        CompletableFuture<JoinGroupResponseData> oldMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(oldMemberResponseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Replace static member with new member id. Old member id should be fenced.
+        CompletableFuture<JoinGroupResponseData> newMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        assertFalse(newMemberResponseFuture.isDone());
+        assertTrue(oldMemberResponseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Complete join for new member.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(newMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), newMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemovePendingMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(0, group.size());
+        assertEquals(1, group.numPendingJoinMembers());
+
+        // Advance clock by session timeout. Pending member should be removed from group as heartbeat expires.
+        context.timer.advanceClock(1000);
+        assertEquals(0, group.numPendingJoinMembers());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemoveMember() throws Exception {
+        // Set initial rebalance delay to simulate a long running rebalance.
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupInitialRebalanceDelayMs(10 * 60 * 1000)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        String memberId = group.leaderOrNull();
+        assertEquals(1, group.size());
+
+        // Advance clock by new member join timeout. Member should be removed from group as heartbeat expires.
+        // A group that transitions to Empty after completing join phase will generate records.
+        context.timer.expectResult(heartbeatKey("group-id", memberId), new ExpectedGenericGroupResult(
+            Collections.singletonList(newGroupMetadataRecord("group-id",
+                new GroupMetadataValue()
+                    .setMembers(Collections.emptyList())
+                    .setGeneration(1)
+                    .setLeader(null)
+                    .setProtocolType("consumer")
+                    .setProtocol(null)
+                    .setCurrentStateTimestamp(context.time.milliseconds()),
+                MetadataVersion.latest())),
+            Errors.NONE,
+            false
+        ));
+
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.size());
+    }
+
+    @Test
+    public void testExistingMemberJoinDeadGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertTrue(group.hasMemberId(memberId));
+
+        group.transitionTo(DEAD);
+
+        response = context.joinGenericGroupAsDynamicMember(request.setMemberId(memberId));
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupExistingPendingMemberWithGroupInstanceIdThrowsException() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertTrue(responseFuture.isDone());
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        assertThrows(IllegalStateException.class,
+            () -> context.sendGenericGroupJoin(request.setMemberId(memberId).setGroupInstanceId("group-instance-id"))
+        );
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        GenericGroupMember member = group.member(memberId);
+
+        assertEquals(protocols, member.supportedProtocols());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(1, group.generationId());
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        // Send updated member metadata. This should trigger a rebalance and complete the join phase.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(memberId)
+            .setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+        assertEquals(protocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingLeaderTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertTrue(group.isLeader(memberId));
+        assertEquals(1, group.generationId());
+
+        group.transitionTo(STABLE);
+        // Sending join group as leader should trigger a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with updated metadata. This should trigger a rebalance.
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        JoinGroupRequestData memberRequest = request.setMemberId(memberId).setProtocols(protocols);
+        memberResponseFuture = context.sendGenericGroupJoin(memberRequest);
+
+        assertFalse(memberResponseFuture.isDone());
+
+        // Leader rejoins. This completes the join group phase.
+        leaderResponseFuture = context.sendGenericGroupJoin(request.setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(3, group.generationId());
+        assertEquals(2, group.size());
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberDoesNotTriggerRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with no metadata changes. This does not trigger a rebalance.
+        memberResponseFuture = context.sendGenericGroupJoin(request.setMemberId(memberId));
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, memberResponseFuture.get(5, TimeUnit.SECONDS).generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberInEmptyState() throws Exception {
+        // Existing member joins a group that is in Empty/Dead state. Ask member to rejoin with generation id reset.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(-1, responseFuture.get(5, TimeUnit.SECONDS).generationId());
+    }
+
+    @Test
+    public void testCompleteJoinRemoveNotYetRejoinedDynamicMembers() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.generationId());
+
+        // Add new member. This triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout. This will expire the leader as it has not rejoined.
+        context.timer.advanceClock(1000);
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertTrue(group.hasMemberId(memberResponseFuture.get(5, TimeUnit.SECONDS).memberId()));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseInEmptyStateSkipsRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+        assertEquals(0, group.generationId());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        group.transitionTo(DEAD);
+
+        // Advance clock by initial rebalance delay to complete join phase.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertEquals(0, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseNoMembersRejoinedExtendsJoinPhase() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("first-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(30000)
+            .withRebalanceTimeoutMs(10000)
+            .build();
+
+        // First member joins group and completes join phase.
+        JoinGroupResponseData firstMemberResponse = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), firstMemberResponse.errorCode());
+        String firstMemberId = firstMemberResponse.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        // Second member joins and group goes into rebalancing state.
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setGroupInstanceId("second-instance-id"));
+
+        // First static member rejoins and completes join phase.
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        String secondMemberId = secondMemberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        // Trigger a rebalance. No members rejoined.
+        context.groupMetadataManager.prepareRebalance(group, "trigger rebalance");
+
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(0, group.numAwaitingJoinResponse());
+
+        // Advance clock by rebalance timeout to complete join phase. As long as both members have not
+        // rejoined, we extend the join phase.
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        // Let first and second member rejoin. This should complete the join phase.
+        firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(secondMemberId)
+            .setGroupInstanceId("second-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(3, group.generationId());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testReplaceStaticMemberInStableStateNoError(
+        boolean supportSkippingAssignment
+    ) throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, supportSkippingAssignment);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String oldMemberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. This should update the log with the generated member id.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request
+                .setProtocols(protocols)
+                .setRebalanceTimeoutMs(7000)
+                .setSessionTimeoutMs(4500),
+            true,
+            supportSkippingAssignment,
+            new ExpectedGenericGroupResult(Errors.NONE, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        String newMemberId = group.staticMemberId("group-instance-id");
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMemberId)
+            .setMemberId(newMemberId)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(supportSkippingAssignment)
+            .setErrorCode(Errors.NONE.code());
+
+        if (supportSkippingAssignment) {
+            expectedResponse
+                .setMembers(Collections.singletonList(
+                    new JoinGroupResponseData.JoinGroupResponseMember()
+                        .setMemberId(newMemberId)
+                        .setGroupInstanceId("group-instance-id")
+                        .setMetadata(protocols.find("range").metadata())
+                    ))
+                .setLeader(newMemberId);
+        }
+
+        GenericGroupMember updatedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        assertEquals(newMemberId, updatedMember.memberId());
+        assertEquals(Optional.of("group-instance-id"), updatedMember.groupInstanceId());
+        assertEquals(7000, updatedMember.rebalanceTimeoutMs());
+        assertEquals(4500, updatedMember.sessionTimeoutMs());
+        assertEquals(protocols, updatedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateWithUpdatedProtocolTriggersRebalance() throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));

Review Comment:
   With the latest changes, i renamed `joinGenericGroup...` to ``joinGenericGroupAsDynamicMemberAndCompleteJoin` and `joinGenericGroupAndCompleteJoin` to make it more explicit.
   
   let me know if this is more readable.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2402,143 +2610,2090 @@ public void testOnNewMetadataImage() {
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = genericGroup(context, "group-id");
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = genericGroup(context, "group-id");
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
         }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+
+        // Send as static member join.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setGroupInstanceId("group-instance-id"), true, true, null);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(firstResponse.leader(), firstMemberId);
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, group.allMembers().iterator().next().memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+        assertNull(response.protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertFalse(responseFuture.isDone());
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+        context.timer.advanceClock(2);
+
+        // Advance clock past initial rebalance delay and verify futures are not completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+
+        // Advance clock beyond recomputed delay and make sure the futures have completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldDelayRebalanceUptoRebalanceTimeout() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs + 1);
+
+        CompletableFuture<JoinGroupResponseData> thirdMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        // Advance clock right before rebalance timeout.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+        assertFalse(thirdMemberResponseFuture.isDone());
+
+        // Advance clock beyond rebalance timeout.
+        context.timer.advanceClock(1);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertTrue(thirdMemberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), thirdMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupReplaceStaticMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .build();
+
+
+        // Send join group as static member.
+        CompletableFuture<JoinGroupResponseData> oldMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(oldMemberResponseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Replace static member with new member id. Old member id should be fenced.
+        CompletableFuture<JoinGroupResponseData> newMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        assertFalse(newMemberResponseFuture.isDone());
+        assertTrue(oldMemberResponseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Complete join for new member.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(newMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), newMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemovePendingMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(0, group.size());
+        assertEquals(1, group.numPendingJoinMembers());
+
+        // Advance clock by session timeout. Pending member should be removed from group as heartbeat expires.
+        context.timer.advanceClock(1000);
+        assertEquals(0, group.numPendingJoinMembers());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemoveMember() throws Exception {
+        // Set initial rebalance delay to simulate a long running rebalance.
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupInitialRebalanceDelayMs(10 * 60 * 1000)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        String memberId = group.leaderOrNull();
+        assertEquals(1, group.size());
+
+        // Advance clock by new member join timeout. Member should be removed from group as heartbeat expires.
+        // A group that transitions to Empty after completing join phase will generate records.
+        context.timer.expectResult(heartbeatKey("group-id", memberId), new ExpectedGenericGroupResult(
+            Collections.singletonList(newGroupMetadataRecord("group-id",
+                new GroupMetadataValue()
+                    .setMembers(Collections.emptyList())
+                    .setGeneration(1)
+                    .setLeader(null)
+                    .setProtocolType("consumer")
+                    .setProtocol(null)
+                    .setCurrentStateTimestamp(context.time.milliseconds()),
+                MetadataVersion.latest())),
+            Errors.NONE,
+            false
+        ));
+
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.size());
+    }
+
+    @Test
+    public void testExistingMemberJoinDeadGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertTrue(group.hasMemberId(memberId));
+
+        group.transitionTo(DEAD);
+
+        response = context.joinGenericGroupAsDynamicMember(request.setMemberId(memberId));
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupExistingPendingMemberWithGroupInstanceIdThrowsException() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertTrue(responseFuture.isDone());
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        assertThrows(IllegalStateException.class,
+            () -> context.sendGenericGroupJoin(request.setMemberId(memberId).setGroupInstanceId("group-instance-id"))
+        );
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        GenericGroupMember member = group.member(memberId);
+
+        assertEquals(protocols, member.supportedProtocols());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(1, group.generationId());
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        // Send updated member metadata. This should trigger a rebalance and complete the join phase.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(memberId)
+            .setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+        assertEquals(protocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingLeaderTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertTrue(group.isLeader(memberId));
+        assertEquals(1, group.generationId());
+
+        group.transitionTo(STABLE);
+        // Sending join group as leader should trigger a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with updated metadata. This should trigger a rebalance.
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        JoinGroupRequestData memberRequest = request.setMemberId(memberId).setProtocols(protocols);
+        memberResponseFuture = context.sendGenericGroupJoin(memberRequest);
+
+        assertFalse(memberResponseFuture.isDone());
+
+        // Leader rejoins. This completes the join group phase.
+        leaderResponseFuture = context.sendGenericGroupJoin(request.setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(3, group.generationId());
+        assertEquals(2, group.size());
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberDoesNotTriggerRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with no metadata changes. This does not trigger a rebalance.
+        memberResponseFuture = context.sendGenericGroupJoin(request.setMemberId(memberId));
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, memberResponseFuture.get(5, TimeUnit.SECONDS).generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberInEmptyState() throws Exception {
+        // Existing member joins a group that is in Empty/Dead state. Ask member to rejoin with generation id reset.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(-1, responseFuture.get(5, TimeUnit.SECONDS).generationId());
+    }
+
+    @Test
+    public void testCompleteJoinRemoveNotYetRejoinedDynamicMembers() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.generationId());
+
+        // Add new member. This triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout. This will expire the leader as it has not rejoined.
+        context.timer.advanceClock(1000);
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertTrue(group.hasMemberId(memberResponseFuture.get(5, TimeUnit.SECONDS).memberId()));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseInEmptyStateSkipsRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+        assertEquals(0, group.generationId());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        group.transitionTo(DEAD);
+
+        // Advance clock by initial rebalance delay to complete join phase.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertEquals(0, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseNoMembersRejoinedExtendsJoinPhase() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("first-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(30000)
+            .withRebalanceTimeoutMs(10000)
+            .build();
+
+        // First member joins group and completes join phase.
+        JoinGroupResponseData firstMemberResponse = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), firstMemberResponse.errorCode());
+        String firstMemberId = firstMemberResponse.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        // Second member joins and group goes into rebalancing state.
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setGroupInstanceId("second-instance-id"));
+
+        // First static member rejoins and completes join phase.
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        String secondMemberId = secondMemberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        // Trigger a rebalance. No members rejoined.
+        context.groupMetadataManager.prepareRebalance(group, "trigger rebalance");
+
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(0, group.numAwaitingJoinResponse());
+
+        // Advance clock by rebalance timeout to complete join phase. As long as both members have not
+        // rejoined, we extend the join phase.
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        // Let first and second member rejoin. This should complete the join phase.
+        firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(secondMemberId)
+            .setGroupInstanceId("second-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(3, group.generationId());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testReplaceStaticMemberInStableStateNoError(
+        boolean supportSkippingAssignment
+    ) throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, supportSkippingAssignment);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String oldMemberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. This should update the log with the generated member id.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request
+                .setProtocols(protocols)
+                .setRebalanceTimeoutMs(7000)
+                .setSessionTimeoutMs(4500),
+            true,
+            supportSkippingAssignment,
+            new ExpectedGenericGroupResult(Errors.NONE, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        String newMemberId = group.staticMemberId("group-instance-id");
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMemberId)
+            .setMemberId(newMemberId)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(supportSkippingAssignment)
+            .setErrorCode(Errors.NONE.code());
+
+        if (supportSkippingAssignment) {
+            expectedResponse
+                .setMembers(Collections.singletonList(
+                    new JoinGroupResponseData.JoinGroupResponseMember()
+                        .setMemberId(newMemberId)
+                        .setGroupInstanceId("group-instance-id")
+                        .setMetadata(protocols.find("range").metadata())
+                    ))
+                .setLeader(newMemberId);
+        }
+
+        GenericGroupMember updatedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        assertEquals(newMemberId, updatedMember.memberId());
+        assertEquals(Optional.of("group-instance-id"), updatedMember.groupInstanceId());
+        assertEquals(7000, updatedMember.rebalanceTimeoutMs());
+        assertEquals(4500, updatedMember.sessionTimeoutMs());
+        assertEquals(protocols, updatedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateWithUpdatedProtocolTriggersRebalance() throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. The selected protocol changes and triggers a rebalance.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array())
+        );
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertEquals(2, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateErrors() throws Exception {
+        // If the append future fails, we need to revert the soft state to the original member.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, false);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        GenericGroupMember oldMember = group.member(response.memberId());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID but the append fails. This reverts the soft state of the group.
+        protocols.add(new JoinGroupRequestProtocol()
+                .setName("roundrobin")
+                .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                    Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setSessionTimeoutMs(6000)
+                .setRebalanceTimeoutMs(7000)
+                .setProtocols(protocols),
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMember.memberId())
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code());
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        GenericGroupMember revertedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(oldMember.memberId(), revertedMember.memberId());
+        assertEquals(oldMember.groupInstanceId(), revertedMember.groupInstanceId());
+        assertEquals(oldMember.rebalanceTimeoutMs(), revertedMember.rebalanceTimeoutMs());
+        assertEquals(oldMember.sessionTimeoutMs(), revertedMember.sessionTimeoutMs());
+        assertEquals(oldMember.supportedProtocols(), revertedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInCompletingRebalanceStateTriggersRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID and triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);

Review Comment:
   replied to comment below.



-- 
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 pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on PR #13870:
URL: https://github.com/apache/kafka/pull/13870#issuecomment-1631769383

   @dajac thanks for the review. I have addressed your 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] dajac commented on a diff in pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }
+
+                result = completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member, group.rebalanceTimeoutMs());
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Void>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;

Review Comment:
   Yeah, it seems that the current implementation is inconsistent. 



-- 
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] CalvinConfluent commented on a diff in pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -102,13 +108,34 @@ public Builder withLoader(CoordinatorLoader<Record> loader) {
             return this;
         }
 
+        public Builder withMetadataImage(MetadataImage metadataImage) {
+            this.metadataImage = metadataImage;
+            return this;
+        }
+
+        public Builder withTime(Time time) {
+            this.time = time;
+            return this;
+        }
+
+        public Builder withTimer(Timer timer) {
+            this.timer = timer;
+            return this;
+        }
+
         public GroupCoordinatorService build() {
             if (config == null)
                 throw new IllegalArgumentException("Config must be set.");
             if (writer == null)
                 throw new IllegalArgumentException("Writer must be set.");
             if (loader == null)
                 throw new IllegalArgumentException("Loader must be set.");
+            if (metadataImage == null)
+                throw new IllegalArgumentException("MetadataImage must be set.");
+            if (time == null)
+                throw new IllegalArgumentException("Time must be set.");
+            if (timer == null)

Review Comment:
   The timer is not used yet. Is it a place holder here?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1362 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,

Review Comment:
   I guess I missed some of the previous discussions, but why it is always either stable or empty when we load a group? Does it mean the rebalancing process will be reverted if the coordinator fails?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1362 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (group.isNew()) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                result = new CoordinatorResult<>(records, appendFuture);
+                genericGroupJoinMember(context, request, group, isUnknownMember, responseFuture);
+            } else {
+                result = genericGroupJoinMember(context, request, group, isUnknownMember, responseFuture);
+            }
+        }
+        return result;
+    }
+
+    private CoordinatorResult<Void, Record> genericGroupJoinMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        boolean isUnknownMember,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+        String joinReason = request.reason();
+        String memberId = request.memberId();
+        if (joinReason == null || joinReason.isEmpty()) {
+            joinReason = "not provided";
+        }
+
+        if (!acceptJoiningMember(group, memberId)) {
+            group.remove(memberId);
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+            );
+
+        } else if (isUnknownMember) {
+            result = genericGroupJoinNewMember(
+                context,
+                request,
+                group,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            result = genericGroupJoinExistingMember(
+                context,
+                request,
+                group,
+                joinReason,
+                responseFuture
+            );
+        }
+
+        tryCompleteJoin(group);
+        return result;
+    }
+
+    private CoordinatorResult<Void, Record> tryCompleteJoin(
+        GenericGroup group
+    ) {
+        // Attempt to complete join group phase. We do not complete
+        // the join group phase if this is the initial rebalance.
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.generationId() != 0
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);

Review Comment:
   Do we need to handle the IllegalStateException(when member id is not known) here and complete the responseFuture here?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1362 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (group.isNew()) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                result = new CoordinatorResult<>(records, appendFuture);
+                genericGroupJoinMember(context, request, group, isUnknownMember, responseFuture);

Review Comment:
   The genericGroupJoinMember can returns a result, why don't we use the return value?



-- 
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] CalvinConfluent commented on a diff in pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1362 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (group.isNew()) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                result = new CoordinatorResult<>(records, appendFuture);
+                genericGroupJoinMember(context, request, group, isUnknownMember, responseFuture);
+            } else {
+                result = genericGroupJoinMember(context, request, group, isUnknownMember, responseFuture);
+            }
+        }
+        return result;
+    }
+
+    private CoordinatorResult<Void, Record> genericGroupJoinMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        boolean isUnknownMember,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+        String joinReason = request.reason();
+        String memberId = request.memberId();
+        if (joinReason == null || joinReason.isEmpty()) {
+            joinReason = "not provided";
+        }
+
+        if (!acceptJoiningMember(group, memberId)) {
+            group.remove(memberId);
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+            );
+
+        } else if (isUnknownMember) {
+            result = genericGroupJoinNewMember(
+                context,
+                request,
+                group,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            result = genericGroupJoinExistingMember(
+                context,
+                request,
+                group,
+                joinReason,
+                responseFuture
+            );
+        }
+
+        tryCompleteJoin(group);
+        return result;
+    }
+
+    private CoordinatorResult<Void, Record> tryCompleteJoin(
+        GenericGroup group
+    ) {
+        // Attempt to complete join group phase. We do not complete
+        // the join group phase if this is the initial rebalance.
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.generationId() != 0
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);

Review Comment:
   Do we need to handle the IllegalStateException(when member id is not known) and complete the responseFuture 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -3022,143 +3242,2087 @@ public void testOnLoaded() {
         assertNotNull(context.timer.timeout(consumerGroupRevocationTimeoutKey("foo", "foo-1")));
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
-
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
-        }
-    }
+    @Test
+    public void testGenerateRecordsOnNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get().errorCode());
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        GenericGroup group = context.createGenericGroup("group-id");
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        assertEquals(
+            Collections.singletonList(RecordHelpers.newEmptyGroupMetadataRecord(group, MetadataVersion.latest())),
+            result.records()
+        );
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
+        context.createGenericGroup("group-id");
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        IntStream.range(0, 10).forEach(i -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+            assertFalse(responseFuture.isDone());
+            assertTrue(result.records().isEmpty());
         });
-        return assignmentMap;
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        GenericGroup group = context.createGenericGroup("group-id");
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> firstRoundFutures = new ArrayList<>();
+        IntStream.range(0, groupMaxSize + 1).forEach(i -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            firstRoundFutures.add(responseFuture);
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture, requiredKnownMemberId);
+            assertTrue(responseFuture.isDone());
+            try {
+                assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get().errorCode());
+            } catch (Exception ignored) {

Review Comment:
   this is required if we want to use the streams api. let me know if we should just use the for each loop



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -299,10 +299,10 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             topicPartitionFor(request.groupId()),
             coordinator -> coordinator.genericGroupJoin(context, request, responseFuture)
         ).exceptionally(exception -> {
-            log.error("Request {} hit an unexpected exception: {}",
-                request, exception.getMessage());
-
             if (!responseFuture.isDone()) {
+                log.error("Request {} hit an unexpected exception: {}",

Review Comment:
   ah makes sense. logging only when it is not a kafka exception 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] dajac commented on a diff in pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -561,7 +625,7 @@ public boolean hasReceivedSyncFromAllMembers() {
      * @return members that have yet to sync.
      */
     public Set<String> allPendingSyncMembers() {
-        return pendingSyncMembers;
+        return new HashSet<>(pendingSyncMembers);

Review Comment:
   This was not addressed.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -177,11 +238,22 @@ GroupMetadataManager build() {
                 consumerGroupMaxSize,
                 consumerGroupSessionTimeoutMs,
                 consumerGroupHeartbeatIntervalMs,
-                consumerGroupMetadataRefreshIntervalMs
+                consumerGroupMetadataRefreshIntervalMs,
+                topicPartition,

Review Comment:
   extremely small nit: Should you move `topicPartition` to the top? This is a common attribute.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -247,31 +324,78 @@ GroupMetadataManager build() {
      */
     private MetadataImage metadataImage;
 
+    /**
+     * An empty result returned to the state machine. This means that
+     * there are no records to append to the log.
+     *
+     * Package private for testing.
+     */
+    static final CoordinatorResult<Void, Record> EMPTY_RESULT =
+        new CoordinatorResult<>(Collections.emptyList(), CompletableFuture.completedFuture(null));

Review Comment:
   nit: I was wondering whether it would make sense to move this to `CoordinatorResult`. We could have a static public constant called `EMPTY`. Then, we could use `CoordinatorResult.EMPTY` in the code. This is a pattern that we already use in a few other places. What do you think?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -247,31 +324,78 @@ GroupMetadataManager build() {
      */
     private MetadataImage metadataImage;
 
+    /**
+     * An empty result returned to the state machine. This means that
+     * there are no records to append to the log.
+     *
+     * Package private for testing.
+     */
+    static final CoordinatorResult<Void, Record> EMPTY_RESULT =
+        new CoordinatorResult<>(Collections.emptyList(), CompletableFuture.completedFuture(null));
+
+    /**
+     * The maximum number of members allowed in a single generic group.
+     */
+    private final int genericGroupMaxSize;
+
+    /**
+     * Initial rebalance delay for members joining a generic group.
+     */
+    private final int genericGroupInitialRebalanceDelayMs;
+
+    /**
+     * The timeout used to wait for a new member in milliseconds.
+     */
+    private final int genericGroupNewMemberJoinTimeoutMs;
+
+    /**
+     * The group minimum session timeout.
+     */
+    private final int genericGroupMinSessionTimeoutMs;
+
+    /**
+     * The group maximum session timeout.
+     */
+    private final int genericGroupMaxSessionTimeoutMs;
+
     private GroupMetadataManager(
         SnapshotRegistry snapshotRegistry,
         LogContext logContext,
         Time time,
-        CoordinatorTimer<Record> timer,
+        CoordinatorTimer<Void, Record> timer,
         List<PartitionAssignor> assignors,
         MetadataImage metadataImage,
         int consumerGroupMaxSize,
         int consumerGroupSessionTimeoutMs,
         int consumerGroupHeartbeatIntervalMs,
-        int consumerGroupMetadataRefreshIntervalMs
+        int consumerGroupMetadataRefreshIntervalMs,
+        TopicPartition topicPartition,
+        int genericGroupMaxSize,
+        int genericGroupInitialRebalanceDelayMs,
+        int genericGroupNewMemberJoinTimeoutMs,
+        int genericGroupMinSessionTimeoutMs,
+        int genericGroupMaxSessionTimeoutMs
     ) {
+        this.logContext = logContext;
         this.log = logContext.logger(GroupMetadataManager.class);
         this.snapshotRegistry = snapshotRegistry;
         this.time = time;
         this.timer = timer;
         this.metadataImage = metadataImage;
         this.assignors = assignors.stream().collect(Collectors.toMap(PartitionAssignor::name, Function.identity()));
+        this.topicPartition = topicPartition;
         this.defaultAssignor = assignors.get(0);
         this.groups = new TimelineHashMap<>(snapshotRegistry, 0);
         this.groupsByTopics = new TimelineHashMap<>(snapshotRegistry, 0);
-        this.consumerGroupMaxSize = consumerGroupMaxSize;
         this.consumerGroupSessionTimeoutMs = consumerGroupSessionTimeoutMs;
         this.consumerGroupHeartbeatIntervalMs = consumerGroupHeartbeatIntervalMs;
         this.consumerGroupMetadataRefreshIntervalMs = consumerGroupMetadataRefreshIntervalMs;
+        this.consumerGroupMaxSize = consumerGroupMaxSize;

Review Comment:
   nit: Could we revert this change?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1243,4 +1406,1283 @@ public static String consumerGroupSessionTimeoutKey(String groupId, String membe
     public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) {
         return "revocation-timeout-" + groupId + "-" + memberId;
     }
+
+     /** Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.

Review Comment:
   nit: The format of the javadoc in is incorrect here.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -231,14 +263,20 @@ public List<Record> build(TopicsImage topicsImage) {
     static class GroupMetadataManagerTestContext {
         static class Builder {
             final private MockTime time = new MockTime();
-            final private MockCoordinatorTimer<Record> timer = new MockCoordinatorTimer<>(time);
+            final private MockCoordinatorTimer<Void, Record> timer = new MockCoordinatorTimer<>(time);
             final private LogContext logContext = new LogContext();
             final private SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
             private MetadataImage metadataImage;
-            private List<PartitionAssignor> assignors;
+            private List<PartitionAssignor> assignors = Collections.singletonList(new MockPartitionAssignor("range"));
             private List<ConsumerGroupBuilder> consumerGroupBuilders = new ArrayList<>();
             private int consumerGroupMaxSize = Integer.MAX_VALUE;
             private int consumerGroupMetadataRefreshIntervalMs = Integer.MAX_VALUE;
+            private final TopicPartition groupMetadataTopicPartition = new TopicPartition("topic", 0);

Review Comment:
   nit: Could we move this next to the other final private attributes? Could we also invest private final to be consistent with the others?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1243,4 +1406,1283 @@ public static String consumerGroupSessionTimeoutKey(String groupId, String membe
     public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) {
         return "revocation-timeout-" + groupId + "-" + memberId;
     }
+
+     /** Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(

Review Comment:
   Is this one covered by a unit test?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -200,13 +272,18 @@ GroupMetadataManager build() {
     /**
      * The system timer.
      */
-    private final CoordinatorTimer<Record> timer;
+    private final CoordinatorTimer<Void, Record> timer;
 
     /**
      * The supported partition assignors keyed by their name.
      */
     private final Map<String, PartitionAssignor> assignors;
 
+    /**
+     * The topic partition associated with the metadata manager.
+     */
+    private final TopicPartition topicPartition;

Review Comment:
   nit: Could you also move this one to the top of the attributes?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2172,8 +2387,7 @@ public void testSubscriptionMetadataRefreshedAgainAfterWriteFailure() {
             .build();
 
         // The metadata refresh flag should be true.
-        ConsumerGroup consumerGroup = context.groupMetadataManager
-            .getOrMaybeCreateConsumerGroup(groupId, false);
+        ConsumerGroup consumerGroup = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);

Review Comment:
   nit: Let's revert this.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -420,11 +501,146 @@ public void assertNoRevocationTimeout(
             String groupId,
             String memberId
         ) {
-            MockCoordinatorTimer.ScheduledTimeout<Record> timeout =
+            ScheduledTimeout<Void, Record> timeout =
                 timer.timeout(consumerGroupRevocationTimeoutKey(groupId, memberId));
             assertNull(timeout);
         }
 
+        GenericGroup createGenericGroup(String groupId) {
+            return groupMetadataManager.getOrMaybeCreateGenericGroup(groupId, true);
+        }
+
+        public CoordinatorResult<Void, Record> sendGenericGroupJoin(
+            JoinGroupRequestData request,
+            CompletableFuture<JoinGroupResponseData> responseFuture
+        ) {
+            return sendGenericGroupJoin(request, responseFuture, false);
+        }
+
+        public CoordinatorResult<Void, Record> sendGenericGroupJoin(
+            JoinGroupRequestData request,
+            CompletableFuture<JoinGroupResponseData> responseFuture,
+            boolean requireKnownMemberId
+        ) {
+            return sendGenericGroupJoin(request, responseFuture, requireKnownMemberId, false);
+        }
+
+        public CoordinatorResult<Void, Record> sendGenericGroupJoin(
+            JoinGroupRequestData request,
+            CompletableFuture<JoinGroupResponseData> responseFuture,
+            boolean requireKnownMemberId,
+            boolean supportSkippingAssignment
+        ) {
+            // requireKnownMemberId is true: version >= 4 (See JoinGroupRequest#requiresKnownMemberId())
+            // supportSkippingAssignment is true: version >= 9 (See JoinGroupRequest#supportsSkippingAssignment())
+            short joinGroupVersion = 3;
+
+            if (requireKnownMemberId) {
+                joinGroupVersion = 4;
+                if (supportSkippingAssignment) {
+                    joinGroupVersion = ApiKeys.JOIN_GROUP.latestVersion();
+                }
+            }
+
+            RequestContext context = new RequestContext(
+                new RequestHeader(
+                    ApiKeys.JOIN_GROUP,
+                    joinGroupVersion,
+                    "client",
+                    0
+                ),
+                "1",
+                InetAddress.getLoopbackAddress(),
+                KafkaPrincipal.ANONYMOUS,
+                ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT),
+                SecurityProtocol.PLAINTEXT,
+                ClientInformation.EMPTY,
+                false
+            );
+
+            return groupMetadataManager.genericGroupJoin(
+                context,
+                request,
+                responseFuture
+            );
+        }
+
+        public JoinGroupResponseData joinGenericGroupAsDynamicMemberAndCompleteJoin(
+            JoinGroupRequestData request
+        ) {
+            boolean requireKnownMemberId = true;
+            String newMemberId = request.memberId();
+
+            try {
+                if (request.memberId().equals(UNKNOWN_MEMBER_ID)) {
+                    // Since member id is required, we need another round to get the successful join group result.
+                    CompletableFuture<JoinGroupResponseData> firstJoinFuture = new CompletableFuture<>();
+                    sendGenericGroupJoin(
+                        request,
+                        firstJoinFuture,
+                        requireKnownMemberId
+                    );
+                    assertTrue(firstJoinFuture.isDone());
+                    assertEquals(Errors.MEMBER_ID_REQUIRED.code(), firstJoinFuture.get().errorCode());
+                    newMemberId = firstJoinFuture.get().memberId();
+                }
+
+                // Second round
+                CompletableFuture<JoinGroupResponseData> secondJoinFuture = new CompletableFuture<>();
+                JoinGroupRequestData secondRequest = new JoinGroupRequestData()
+                    .setGroupId(request.groupId())
+                    .setMemberId(newMemberId)
+                    .setProtocolType(request.protocolType())
+                    .setProtocols(request.protocols())
+                    .setSessionTimeoutMs(request.sessionTimeoutMs())
+                    .setRebalanceTimeoutMs(request.rebalanceTimeoutMs())
+                    .setReason(request.reason());
+
+                sendGenericGroupJoin(
+                    secondRequest,
+                    secondJoinFuture,
+                    requireKnownMemberId
+                );
+
+                List<ExpiredTimeout<Void, Record>> timeouts = sleep(genericGroupInitialRebalanceDelayMs);
+                assertEquals(1, timeouts.size());
+                assertTrue(secondJoinFuture.isDone());
+                assertEquals(Errors.NONE.code(), secondJoinFuture.get().errorCode());
+                return secondJoinFuture.get();
+            } catch (Exception e) {
+                fail("Failed to due: " + e.getMessage());
+            }

Review Comment:
   nit: What the reason for this? If you don't catch it, the test will also fail.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2944,17 +3161,20 @@ public void testRevocationTimeoutExpiration() {
         );
 
         // Advance time past the revocation timeout.
-        List<MockCoordinatorTimer.ExpiredTimeout<Record>> timeouts = context.sleep(10000 + 1);
+        List<ExpiredTimeout<Void, Record>> timeouts = context.sleep(10000 + 1);
 
         // Verify the expired timeout.
         assertEquals(
-            Collections.singletonList(new MockCoordinatorTimer.ExpiredTimeout<Record>(
+            Collections.singletonList(new ExpiredTimeout<Void, Record>(
                 consumerGroupRevocationTimeoutKey(groupId, memberId1),
-                Arrays.asList(
-                    RecordHelpers.newCurrentAssignmentTombstoneRecord(groupId, memberId1),
-                    RecordHelpers.newTargetAssignmentTombstoneRecord(groupId, memberId1),
-                    RecordHelpers.newMemberSubscriptionTombstoneRecord(groupId, memberId1),
-                    RecordHelpers.newGroupEpochRecord(groupId, 3)
+                new CoordinatorResult<>(
+                    Arrays.asList(
+                        RecordHelpers.newCurrentAssignmentTombstoneRecord(groupId, memberId1),
+                        RecordHelpers.newTargetAssignmentTombstoneRecord(groupId, memberId1),
+                        RecordHelpers.newMemberSubscriptionTombstoneRecord(groupId, memberId1),
+                        RecordHelpers.newGroupEpochRecord(groupId, 3)
+                    ),
+                    null

Review Comment:
   nit: `null` could be removed.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -376,19 +448,28 @@ public CoordinatorResult<ConsumerGroupHeartbeatResponseData, Record> consumerGro
             return result;
         }
 
-        public List<MockCoordinatorTimer.ExpiredTimeout<Record>> sleep(long ms) {
+        public List<ExpiredTimeout<Void, Record>> sleep(long ms) {
             time.sleep(ms);
-            List<MockCoordinatorTimer.ExpiredTimeout<Record>> timeouts = timer.poll();
-            timeouts.forEach(timeout -> timeout.records.forEach(this::replay));
+            List<ExpiredTimeout<Void, Record>> timeouts = timer.poll();
+            timeouts.forEach(timeout -> {
+                if (timeout.result.replayRecords()) {
+                    timeout.result.records().forEach(this::replay);
+                }
+            });
             return timeouts;
         }
 
-        public MockCoordinatorTimer.ScheduledTimeout<Record> assertSessionTimeout(
+        public void sleepAndAssertEmptyResult(long ms) {
+            List<ExpiredTimeout<Void, Record>> timeouts = sleep(ms);
+            timeouts.forEach(timeout -> assertEquals(EMPTY_RESULT, timeout.result));
+        }

Review Comment:
   It seems based on the usages of this method that only one timeouts is expected all the time. Should we enforce it as well?
   
   More generally, I was wondering if having a `assertEmptyTimeout` helper method and using `assertEmptyTimeout(context.sleep(...))` would have a better separation of concerns. I leave this up to you.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -3022,143 +3242,2087 @@ public void testOnLoaded() {
         assertNotNull(context.timer.timeout(consumerGroupRevocationTimeoutKey("foo", "foo-1")));
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
-
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
-        }
-    }
+    @Test
+    public void testGenerateRecordsOnNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get().errorCode());
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        GenericGroup group = context.createGenericGroup("group-id");
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        assertEquals(
+            Collections.singletonList(RecordHelpers.newEmptyGroupMetadataRecord(group, MetadataVersion.latest())),
+            result.records()
+        );
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
+        context.createGenericGroup("group-id");
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        IntStream.range(0, 10).forEach(i -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+            assertFalse(responseFuture.isDone());
+            assertTrue(result.records().isEmpty());
         });
-        return assignmentMap;
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        GenericGroup group = context.createGenericGroup("group-id");
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> firstRoundFutures = new ArrayList<>();
+        IntStream.range(0, groupMaxSize + 1).forEach(i -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            firstRoundFutures.add(responseFuture);
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture, requiredKnownMemberId);
+            assertTrue(responseFuture.isDone());
+            try {
+                assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get().errorCode());
+            } catch (Exception ignored) {

Review Comment:
   This catch is a bit suspicious here. I suppose that it would also catch the error thrown by `assertEquals`, no?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -3022,143 +3242,2087 @@ public void testOnLoaded() {
         assertNotNull(context.timer.timeout(consumerGroupRevocationTimeoutKey("foo", "foo-1")));
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
-
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
-        }
-    }
+    @Test
+    public void testGenerateRecordsOnNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get().errorCode());
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        GenericGroup group = context.createGenericGroup("group-id");
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        assertEquals(
+            Collections.singletonList(RecordHelpers.newEmptyGroupMetadataRecord(group, MetadataVersion.latest())),
+            result.records()
+        );
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
+        context.createGenericGroup("group-id");
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        IntStream.range(0, 10).forEach(i -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+            assertFalse(responseFuture.isDone());
+            assertTrue(result.records().isEmpty());
         });
-        return assignmentMap;
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        GenericGroup group = context.createGenericGroup("group-id");
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> firstRoundFutures = new ArrayList<>();
+        IntStream.range(0, groupMaxSize + 1).forEach(i -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            firstRoundFutures.add(responseFuture);
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture, requiredKnownMemberId);
+            assertTrue(responseFuture.isDone());
+            try {
+                assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get().errorCode());
+            } catch (Exception ignored) {
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
-        }
+            assertTrue(result.records().isEmpty());
+        });
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(firstRoundFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        List<CompletableFuture<JoinGroupResponseData>> secondRoundFutures = new ArrayList<>();
+        memberIds.forEach(memberId -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            secondRoundFutures.add(responseFuture);
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(
+                request.setMemberId(memberId),
+                responseFuture,
+                requiredKnownMemberId
+            );
+            assertTrue(result.records().isEmpty());
+        });
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.sleepAndAssertEmptyResult(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.sleepAndAssertEmptyResult(50);
+
+        verifyGenericGroupJoinResponses(secondRoundFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        List<CompletableFuture<JoinGroupResponseData>> thirdRoundFutures = new ArrayList<>();
+        memberIds.forEach(memberId -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            thirdRoundFutures.add(responseFuture);
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(
+                request.setMemberId(memberId),
+                responseFuture,
+                requiredKnownMemberId
+            );
+            assertTrue(result.records().isEmpty());
+        });
+
+        verifyGenericGroupJoinResponses(thirdRoundFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
-        }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        GenericGroup group = context.createGenericGroup("group-id");
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> firstRoundFutures = new ArrayList<>();
+        IntStream.range(0, groupMaxSize + 1).forEach(i -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            firstRoundFutures.add(responseFuture);
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture, requiredKnownMemberId);
+            assertTrue(result.records().isEmpty());
+        });
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.sleepAndAssertEmptyResult(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.sleepAndAssertEmptyResult(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(firstRoundFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        List<CompletableFuture<JoinGroupResponseData>> secondRoundFutures = new ArrayList<>();
+        memberIds.forEach(memberId -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            secondRoundFutures.add(responseFuture);
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(
+                request.setMemberId(memberId),
+                responseFuture,
+                requiredKnownMemberId
+            );
+            assertTrue(result.records().isEmpty());
+        });
+
+        verifyGenericGroupJoinResponses(secondRoundFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-        assertEquals(expected.version(), actual.version());
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.createGenericGroup("group-id");
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
-        }
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> firstRoundFutures = new ArrayList<>();
+        groupInstanceIds.forEach(instanceId -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            firstRoundFutures.add(responseFuture);
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request.setGroupInstanceId(instanceId), responseFuture);
+            assertTrue(result.records().isEmpty());
+        });
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.sleepAndAssertEmptyResult(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.sleepAndAssertEmptyResult(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(firstRoundFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        List<CompletableFuture<JoinGroupResponseData>> secondRoundFutures = new ArrayList<>();
+        IntStream.range(0, groupMaxSize + 1).forEach(i -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            secondRoundFutures.add(responseFuture);
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(
+                request.setMemberId(memberIds.get(i))
+                    .setGroupInstanceId(groupInstanceIds.get(i)),
+                responseFuture
+            );
+            assertTrue(result.records().isEmpty());
+        });
+
+        verifyGenericGroupJoinResponses(secondRoundFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        GenericGroup group = context.createGenericGroup("group-id");
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. Generate member ids.
+        List<CompletableFuture<JoinGroupResponseData>> firstRoundFutures = new ArrayList<>();
+        IntStream.range(0, groupMaxSize + 1).forEach(i -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            firstRoundFutures.add(responseFuture);
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(
+                request,
+                responseFuture,
+                requiredKnownMemberId
+            );
+            assertTrue(result.records().isEmpty());
         });
-        return assignmentMap;
+
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(firstRoundFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        memberIds.forEach(memberId -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(
+                request.setMemberId(memberId),
+                responseFuture,
+                requiredKnownMemberId
+            );
+            assertTrue(result.records().isEmpty());
+        });
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        List<CompletableFuture<JoinGroupResponseData>> thirdRoundFutures = new ArrayList<>();
+        memberIds.forEach(memberId -> {
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            thirdRoundFutures.add(responseFuture);
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(
+                request.setMemberId(memberId),
+                responseFuture,
+                requiredKnownMemberId
+            );
+            assertTrue(result.records().isEmpty());
+        });
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.sleepAndAssertEmptyResult(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.sleepAndAssertEmptyResult(50);
+
+        verifyGenericGroupJoinResponses(thirdRoundFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
-}
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.createGenericGroup("group-id");
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+            responseFutures.add(responseFuture);
+            CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+            assertTrue(result.records().isEmpty());
+        });
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.sleepAndAssertEmptyResult(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+        assertTrue(responseFuture.isDone());
+        assertTrue(result.records().isEmpty());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownMemberNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get().errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        responseFuture = new CompletableFuture<>();
+        result = context.sendGenericGroupJoin(request, responseFuture);
+
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+        context.createGenericGroup("group-id");
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request);
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+        context.createGenericGroup("group-id");
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+
+        // Send as static member join.
+        responseFuture = new CompletableFuture<>();
+        result = context.sendGenericGroupJoin(request.setGroupInstanceId("group-instance-id"), responseFuture, true, true);
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+        context.createGenericGroup("group-id");
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.createGenericGroup("group-id");
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(firstResponse.leader(), firstMemberId);
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000),
+            secondResponseFuture
+        );
+        assertTrue(result.records().isEmpty());
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.sleepAndAssertEmptyResult(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+        JoinGroupResponseData secondResponse = secondResponseFuture.get();
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, group.allMembers().iterator().next().memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+        context.createGenericGroup("group-id");
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+
+        assertTrue(result.records().isEmpty());
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = new CompletableFuture<>();
+        result = context.sendGenericGroupJoin(request.setProtocols(otherProtocols), otherResponseFuture);
+
+        assertTrue(result.records().isEmpty());
+        assertTrue(otherResponseFuture.isDone());
+
+        context.sleepAndAssertEmptyResult(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get().errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+        context.createGenericGroup("group-id");
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture, true);
+
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = new CompletableFuture<>();
+        result = context.sendGenericGroupJoin(request, responseFuture, true);
+
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = new CompletableFuture<>();
+        result = context.sendGenericGroupJoin(request.setProtocols(protocols), responseFuture, true);
+
+        assertTrue(result.records().isEmpty());
+        assertFalse(responseFuture.isDone());
+
+        context.sleepAndAssertEmptyResult(context.genericGroupInitialRebalanceDelayMs);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+        context.createGenericGroup("group-id");
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        context.joinGenericGroupAndCompleteJoin(request, false, true);
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+        context.createGenericGroup("group-id");
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAndCompleteJoin(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(
+            request.setMemberId("unknown-member-id"),
+            responseFuture
+        );
+
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+        context.createGenericGroup("group-id");
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMemberAndCompleteJoin(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request
+            .setMemberId("other-member-id"), responseFuture);
+
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+        GenericGroup group = context.createGenericGroup("group-id");
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, responseFuture);
+
+        assertTrue(result.records().isEmpty());
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get().errorCode());
+        assertNull(responseFuture.get().protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+        context.createGenericGroup("group-id");
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = new CompletableFuture<>();
+        CoordinatorResult<Void, Record> result = context.sendGenericGroupJoin(request, leaderResponseFuture);
+
+        assertTrue(result.records().isEmpty());
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = new CompletableFuture<>();
+        result = context.sendGenericGroupJoin(request, memberResponseFuture);
+
+        assertTrue(result.records().isEmpty());
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.sleepAndAssertEmptyResult(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {

Review Comment:
   nit: While we are here, would it make to normalize the name of all tests starting with `should`? They should ideally start with `test...` like the others.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -319,6 +443,45 @@ ConsumerGroup getOrMaybeCreateConsumerGroup(
         }
     }
 
+    /**
+     * Gets or maybe creates a generic group.
+     *
+     * @param groupId           The group id.
+     * @param createIfNotExists A boolean indicating whether the group should be
+     *                          created if it does not exist.
+     *
+     * @return A GenericGroup.
+     * @throws GroupIdNotFoundException if the group does not exist and createIfNotExists is false or

Review Comment:
   nit: Add javadoc for UnknownMemberIdException.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +282,32 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isGroupIdNotEmpty(request.groupId())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleWriteOperation("generic-group-join",
+            topicPartitionFor(request.groupId()),
+            coordinator -> coordinator.genericGroupJoin(context, request, responseFuture)
+        ).exceptionally(exception -> {
+            log.error("Request {} hit an unexpected exception: {}",
+                request, exception.getMessage());

Review Comment:
   Yeah, I think that it depends on what we mean by unexpected. I would remove it for now given that we also log something when the append future fails. We can always bring it back later if needed.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -579,4 +618,32 @@ public void shutdown() {
         Utils.closeQuietly(runtime, "coordinator runtime");
         log.info("Shutdown complete.");
     }
+
+    private static boolean isGroupIdNotEmpty(String groupId) {
+        return groupId != null && !groupId.isEmpty();
+    }
+
+    private static Errors toResponseError(Errors appendError) {

Review Comment:
   That seems reasonable. We can see later if we could also share this logic with the consumer group heartbeat handling.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1043,4 +1221,1331 @@ public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
             }
         });
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",

Review Comment:
   Correct. I think that you saw that in my other PR. The issue with logging here is that it will log state metadata as well and we don't want this.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -798,7 +961,7 @@ private void scheduleConsumerGroupSessionTimeout(
                 ConsumerGroupMember member = group.getOrMaybeCreateMember(memberId, false);
                 log.info("[GroupId {}] Member {} fenced from the group because its session expired.",
                     groupId, memberId);
-                return consumerGroupFenceMember(group, member);
+                return new CoordinatorResult<>(consumerGroupFenceMember(group, member), null);

Review Comment:
   nit: There is a constructor which does not take the response. We could use it and remove `null` here. There are a few other cases. I won't mention them again.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1243,4 +1406,1283 @@ public static String consumerGroupSessionTimeoutKey(String groupId, String membe
     public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) {
         return "revocation-timeout-" + groupId + "-" + memberId;
     }
+
+     /** Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if a group was newly created, we need to append
+                // records to the log to commit the group to the timeline data structure.
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());

Review Comment:
   I was wondering if it would be better to complete the future directly here as well in order to be consistent. I think that you did this in the sync handling if I understood you correctly. What do you think?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -279,12 +337,20 @@ public GroupMetadataManagerTestContext build() {
                         .withTime(time)
                         .withTimer(timer)
                         .withMetadataImage(metadataImage)
+                        .withTopicPartition(groupMetadataTopicPartition)
                         .withConsumerGroupHeartbeatInterval(5000)
                         .withConsumerGroupSessionTimeout(45000)
                         .withConsumerGroupMaxSize(consumerGroupMaxSize)
-                        .withAssignors(assignors)
                         .withConsumerGroupMetadataRefreshIntervalMs(consumerGroupMetadataRefreshIntervalMs)
-                        .build()
+                        .withGenericGroupMaxSize(genericGroupMaxSize)
+                        .withGenericGroupMinSessionTimeoutMs(genericGroupMinSessionTimeoutMs)
+                        .withGenericGroupMaxSessionTimeoutMs(genericGroupMaxSessionTimeoutMs)
+                        .withGenericGroupInitialRebalanceDelayMs(genericGroupInitialRebalanceDelayMs)
+                        .withGenericGroupNewMemberJoinTimeoutMs(genericGroupNewMemberJoinTimeoutMs)
+                        .withAssignors(assignors)

Review Comment:
   nit: Could we move this one back to its original place?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2062,8 +2278,7 @@ public void testSubscriptionMetadataRefreshedAfterGroupIsLoaded() {
             .build();
 
         // The metadata refresh flag should be true.
-        ConsumerGroup consumerGroup = context.groupMetadataManager
-            .getOrMaybeCreateConsumerGroup(groupId, false);
+        ConsumerGroup consumerGroup = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);

Review Comment:
   nit: Let's revert this.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java:
##########
@@ -271,4 +289,163 @@ public void testOnResignation() {
             10
         );
     }
+
+    @Test
+    public void testJoinGroup() {
+        CoordinatorRuntime<ReplicatedGroupCoordinator, Record> runtime = mockRuntime();
+        GroupCoordinatorService service = new GroupCoordinatorService(
+            new LogContext(),
+            createConfig(),
+            runtime
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestData()
+            .setGroupId("foo");
+
+        service.startup(() -> 1);
+
+        when(runtime.scheduleWriteOperation(
+            ArgumentMatchers.eq("generic-group-join"),
+            ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
+            ArgumentMatchers.any()
+        )).thenReturn(CompletableFuture.completedFuture(
+            new JoinGroupResponseData()
+        ));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = service.joinGroup(
+            requestContext(ApiKeys.JOIN_GROUP),
+            request,
+            BufferSupplier.NO_CACHING
+        );
+
+        assertFalse(responseFuture.isDone());
+    }
+
+    @Test
+    public void testJoinGroupWithException() throws Exception {
+        CoordinatorRuntime<ReplicatedGroupCoordinator, Record> runtime = mockRuntime();
+        GroupCoordinatorService service = new GroupCoordinatorService(
+            new LogContext(),
+            createConfig(),
+            runtime
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestData()
+            .setGroupId("foo");
+
+        service.startup(() -> 1);
+
+        when(runtime.scheduleWriteOperation(
+            ArgumentMatchers.eq("generic-group-join"),
+            ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
+            ArgumentMatchers.any()
+        )).thenReturn(FutureUtils.failedFuture(new IllegalStateException()));
+
+        CompletableFuture<JoinGroupResponseData> future = service.joinGroup(
+            requestContext(ApiKeys.JOIN_GROUP),
+            request,
+            BufferSupplier.NO_CACHING
+        );
+
+        assertEquals(
+            new JoinGroupResponseData()
+                .setErrorCode(Errors.UNKNOWN_SERVER_ERROR.code()),
+            future.get(5, TimeUnit.SECONDS)
+        );
+    }
+
+    @Test
+    public void testAppendErrorConversion() throws Exception {

Review Comment:
   nit: `testJoinGroupAppend...`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1243,4 +1406,1283 @@ public static String consumerGroupSessionTimeoutKey(String groupId, String membe
     public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) {
         return "revocation-timeout-" + groupId + "-" + memberId;
     }
+
+     /** Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if a group was newly created, we need to append
+                // records to the log to commit the group to the timeline data structure.
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newEmptyGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Attempt to complete join group phase. We do not complete
+     * the join group phase if this is the initial rebalance.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinPhase(GenericGroup group) {
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                responseFuture
+            );
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (JoinGroupRequest.requiresKnownMemberId(context.apiVersion())) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberThenRebalanceOrCompleteJoin(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " + group.stateAsString() +
+                            "; client reason: " + JoinGroupRequest.joinReason(request),
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(group.isLeader(memberId) ?
+                                group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}) with {} members",
+                    groupId, group.generationId(), topicPartition, group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(heartbeatKey(groupId, member.memberId()));
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            false,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalanceOrCompleteJoin(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member. Then begin a rebalance or complete the join phase.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateMemberThenRebalanceOrCompleteJoin(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalanceOrCompleteJoin(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.previousState() == EMPTY) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalanceOrCompleteJoin(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + JoinGroupRequest.joinReason(request));
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state. Otherwise, try
+     * to complete the join phase.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybePrepareRebalanceOrCompleteJoin(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        } else {
+            return maybeCompleteJoinPhase(group);
+        }
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     *
+     * Package private for testing.
+     */
+    CoordinatorResult<Void, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.isInState(EMPTY);
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : maybeCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if (!member.hasAssignment() && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            false,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<Void, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {

Review Comment:
   Note: We will have to also verify the number of member after the group is loaded, I think. This is something for another PR but to keep in mind.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2561,18 +2775,21 @@ public void testSessionTimeoutExpiration() {
         context.assertSessionTimeout(groupId, memberId, 45000);
 
         // Advance time past the session timeout.
-        List<MockCoordinatorTimer.ExpiredTimeout<Record>> timeouts = context.sleep(45000 + 1);
+        List<ExpiredTimeout<Void, Record>> timeouts = context.sleep(45000 + 1);
 
         // Verify the expired timeout.
         assertEquals(
-            Collections.singletonList(new MockCoordinatorTimer.ExpiredTimeout<Record>(
+            Collections.singletonList(new ExpiredTimeout<Void, Record>(
                 consumerGroupSessionTimeoutKey(groupId, memberId),
-                Arrays.asList(
-                    RecordHelpers.newCurrentAssignmentTombstoneRecord(groupId, memberId),
-                    RecordHelpers.newTargetAssignmentTombstoneRecord(groupId, memberId),
-                    RecordHelpers.newMemberSubscriptionTombstoneRecord(groupId, memberId),
-                    RecordHelpers.newGroupSubscriptionMetadataRecord(groupId, Collections.emptyMap()),
-                    RecordHelpers.newGroupEpochRecord(groupId, 2)
+                new CoordinatorResult<>(
+                    Arrays.asList(
+                        RecordHelpers.newCurrentAssignmentTombstoneRecord(groupId, memberId),
+                        RecordHelpers.newTargetAssignmentTombstoneRecord(groupId, memberId),
+                        RecordHelpers.newMemberSubscriptionTombstoneRecord(groupId, memberId),
+                        RecordHelpers.newGroupSubscriptionMetadataRecord(groupId, Collections.emptyMap()),
+                        RecordHelpers.newGroupEpochRecord(groupId, 2)
+                    ),
+                    null

Review Comment:
   nit: `null` could be removed.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -1243,4 +1406,1283 @@ public static String consumerGroupSessionTimeoutKey(String groupId, String membe
     public static String consumerGroupRevocationTimeoutKey(String groupId, String memberId) {
         return "revocation-timeout-" + groupId + "-" + memberId;
     }
+
+     /** Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should be removed.
+            groups.remove(groupId);
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = member.rebalanceTimeout() == -1 ?
+                    member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            boolean isNewGroup = !groups.containsKey(groupId);
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    responseFuture
+                );
+            }
+
+            if (isNewGroup && result == EMPTY_RESULT) {
+                // If there are no records to append and if a group was newly created, we need to append
+                // records to the log to commit the group to the timeline data structure.
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newEmptyGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+            }
+        }
+        return result;
+    }
+
+    /**
+     * Attempt to complete join group phase. We do not complete
+     * the join group phase if this is the initial rebalance.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinPhase(GenericGroup group) {
+        if (group.isInState(PREPARING_REBALANCE) &&
+            group.hasAllMembersJoined() &&
+            group.previousState() != EMPTY
+        ) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                responseFuture
+            );
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        if (JoinGroupRequest.requiresKnownMemberId(context.apiVersion())) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), request.protocols())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberThenRebalanceOrCompleteJoin(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " + group.stateAsString() +
+                            "; client reason: " + JoinGroupRequest.joinReason(request),
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(group.isLeader(memberId) ?
+                                group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberThenRebalanceOrCompleteJoin(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " + group.stateAsString() +
+                                "; client reason: " + JoinGroupRequest.joinReason(request),
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}) with {} members",
+                    groupId, group.generationId(), topicPartition, group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    timer.cancel(heartbeatKey(groupId, member.memberId()));
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            false,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalanceOrCompleteJoin(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member. Then begin a rebalance or complete the join phase.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateMemberThenRebalanceOrCompleteJoin(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalanceOrCompleteJoin(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.previousState() == EMPTY) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalanceOrCompleteJoin(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + JoinGroupRequest.joinReason(request));
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state. Otherwise, try
+     * to complete the join phase.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybePrepareRebalanceOrCompleteJoin(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        } else {
+            return maybeCompleteJoinPhase(group);
+        }
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     *
+     * Package private for testing.
+     */
+    CoordinatorResult<Void, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.isInState(EMPTY);
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : maybeCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> maybeCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                false,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if (!member.hasAssignment() && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            false,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<Void, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        switch (group.currentState()) {
+            case EMPTY:
+            case DEAD:
+                // Always accept the request when the group is empty or dead
+                return true;
+            case PREPARING_REBALANCE:
+                // An existing member is accepted if it is already awaiting. New members are accepted
+                // up to the max group size. Note that the number of awaiting members is used here
+                // for two reasons:
+                // 1) the group size is not reliable as it could already be above the max group size
+                //    if the max group size was reduced.
+                // 2) using the number of awaiting members allows to kick out the last rejoining
+                //    members of the group.
+                return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                    group.numAwaitingJoinResponse() < genericGroupMaxSize;
+            case COMPLETING_REBALANCE:
+            case STABLE:
+                // An existing member is accepted. New members are accepted up to the max group size.
+                // Note that the group size is used here. When the group transitions to CompletingRebalance,
+                // members who haven't rejoined are removed.
+                return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+            default:
+                throw new IllegalStateException("Unknown group state: " + group.stateAsString());
+        }
+    }
+
+    /**
+     * Update a static member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group of the static member.
+     * @param oldMemberId     The existing static member id.
+     * @param newMemberId     The new joining static member id.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<Void, Record> updateStaticMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String oldMemberId,
+        String newMemberId,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String currentLeader = group.leaderOrNull();
+        GenericGroupMember newMember = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+
+        // Heartbeat of old member id will expire without effect since the group no longer contains that member id.
+        // New heartbeat shall be scheduled with new member id.
+        rescheduleGenericGroupMemberHeartbeat(group, newMember);
+
+        int oldRebalanceTimeoutMs = newMember.rebalanceTimeoutMs();
+        int oldSessionTimeoutMs = newMember.sessionTimeoutMs();
+        JoinGroupRequestProtocolCollection oldProtocols = newMember.supportedProtocols();
+
+        group.updateMember(
+            newMember,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        if (group.isInState(STABLE)) {
+            // Check if group's selected protocol of next generation will change, if not, simply store group to persist
+            // the updated static member, if yes, rebalance should be triggered to keep the group's assignment
+            // and selected protocol consistent
+            String groupInstanceId = request.groupInstanceId();
+            String selectedProtocolForNextGeneration = group.selectProtocol();
+            if (group.protocolName().orElse("").equals(selectedProtocolForNextGeneration)) {
+                log.info("Static member which joins during Stable stage and doesn't affect " +
+                    "the selected protocol will not trigger a rebalance.");
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        log.warn("Failed to persist metadata for group {} static member {} with " +
+                            "group instance id {} due to {}. Reverting to old member id {}.",
+                            group.groupId(), newMemberId, groupInstanceId, t.getMessage(), oldMemberId);
+
+                        // Failed to persist the member id of the given static member, revert the update of the static member in the group.
+                        group.updateMember(newMember, oldProtocols, oldRebalanceTimeoutMs, oldSessionTimeoutMs, null);
+                        GenericGroupMember oldMember = group.replaceStaticMember(groupInstanceId, newMemberId, oldMemberId);
+                        rescheduleGenericGroupMemberHeartbeat(group, oldMember);
+
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setMembers(Collections.emptyList())
+                                .setMemberId(UNKNOWN_MEMBER_ID)
+                                .setGenerationId(group.generationId())
+                                .setProtocolName(group.protocolName().orElse(null))
+                                .setProtocolType(group.protocolType().orElse(null))
+                                .setLeader(currentLeader)
+                                .setSkipAssignment(false)
+                                .setErrorCode(Errors.forException(t).code())
+                        );
+                    } else if (JoinGroupRequest.supportsSkippingAssignment(context.apiVersion())) {
+                        boolean isLeader = group.isLeader(newMemberId);
+
+                        group.completeJoinFuture(newMember, new JoinGroupResponseData()
+                            .setMembers(isLeader ? group.currentGenericGroupMembers() : Collections.emptyList())
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(isLeader)
+                        );
+                    } else {
+                        group.completeJoinFuture(newMember, new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(currentLeader)
+                            .setSkipAssignment(false)
+                        );
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+            } else {
+                return maybePrepareRebalanceOrCompleteJoin(
+                    group,
+                    "Group's selectedProtocol will change because static member " +
+                        newMember.memberId() + " with instance id " + groupInstanceId +
+                        " joined with change of protocol; client reason: " + JoinGroupRequest.joinReason(request)
+                );
+            }
+        } else if (group.isInState(COMPLETING_REBALANCE)) {
+            // if the group is in after-sync stage, upon getting a new join-group of a known static member
+            // we should still trigger a new rebalance, since the old member may already be sent to the leader
+            // for assignment, and hence when the assignment gets back there would be a mismatch of the old member id
+            // with the new replaced member id. As a result the new member id would not get any assignment.
+            return prepareRebalance(
+                group,
+                "Updating metadata for static member " + newMember.memberId() + " with instance id " +
+                    request.groupInstanceId() + "; client reason: " + JoinGroupRequest.joinReason(request)
+            );
+        } else if (group.isInState(EMPTY) || group.isInState(DEAD)) {
+            throw new IllegalStateException("Group " + group.groupId() + " was not supposed to be in the state " +
+                group.stateAsString() + " when the unknown static member " + request.groupInstanceId() + " rejoins.");
+
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Generate a heartbeat key for the timer.
+     *
+     * Package private for testing.
+     *
+     * @param groupId   The group id.
+     * @param memberId  The member id.
+     *
+     * @return the heartbeat key.
+     */
+    static String heartbeatKey(String groupId, String memberId) {

Review Comment:
   nit: Could we prefix this one and the two others with `genericGroup`?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +282,32 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isGroupIdNotEmpty(request.groupId())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleWriteOperation("generic-group-join",
+            topicPartitionFor(request.groupId()),
+            coordinator -> coordinator.genericGroupJoin(context, request, responseFuture)
+        ).exceptionally(exception -> {
+            log.error("Request {} hit an unexpected exception: {}",
+                request, exception.getMessage());
+
+            if (!responseFuture.isDone()) {
+                Errors clientError = toResponseError(Errors.forException(exception));

Review Comment:
   That makes sense.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/generic/GenericGroup.java:
##########
@@ -561,7 +625,7 @@ public boolean hasReceivedSyncFromAllMembers() {
      * @return members that have yet to sync.
      */
     public Set<String> allPendingSyncMembers() {
-        return pendingSyncMembers;
+        return new HashSet<>(pendingSyncMembers);

Review Comment:
   Would it make sense to make the copy on the other side? It is a bit weird to anticipate this here because we don't do this for other accessors. I am usually tempted to return unmodifiable collections in the case to prevent this kind of issue.



-- 
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 pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on PR #13870:
URL: https://github.com/apache/kafka/pull/13870#issuecomment-1641304041

   the test failures are unrelated


-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);

Review Comment:
   i don't think this fits well because we may have other records to append in this method. As mentioned above, I will store the generic groups in a separate hash map to prevent this from happening.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1045,1265 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    // Below stores all methods to handle generic group APIs.
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Errors>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < groupMinSessionTimeoutMs ||
+            sessionTimeoutMs > groupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = (GenericGroup) getOrMaybeCreateGroup(groupId, GENERIC, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(request.protocolType(), GenericGroupMember.plainProtocolSet(protocols))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4
+            && (request.groupInstanceId() == null || request.groupInstanceId().isEmpty());
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata())).collect(Collectors.toList());
+
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(protocols))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(protocols)) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(protocols)) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            return tryCompleteJoinElseSchedule(group);
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Errors> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((error, t) -> {
+                    if (t != null || error != Errors.NONE) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        String message = t != null ? t.getMessage() : error.message();
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), message);
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member);
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = request.protocols().stream()
+            .map(protocol -> new Protocol(protocol.name(), protocol.metadata()))
+            .collect(Collectors.toList());
+
+        group.updateMember(member, protocols, request.rebalanceTimeoutMs(), request.sessionTimeoutMs(), responseFuture);
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Errors>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        List<Protocol> protocols = new ArrayList<>();
+        request.protocols().forEach(protocol -> protocols.add(new Protocol(protocol.name(), protocol.metadata())));
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            protocols
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, newMemberJoinTimeoutMs);

Review Comment:
   are you suggesting a different key? what would be the benefit?



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntime.java:
##########
@@ -732,6 +771,117 @@ public String toString() {
         }
     }
 
+    class GenericGroupEvent implements CoordinatorEvent {
+        /**
+         * The topic partition that this internal event is applied to.
+         */
+        final TopicPartition tp;
+
+        /**
+         * The operation name.
+         */
+        final String name;
+
+        /**
+         * The internal operation to execute.
+         */
+        final GenericGroupOperation<U> op;
+
+        /**
+         * The result of the generic group operation. It could be null
+         * if an exception is thrown before it is assigned.
+         */
+        CoordinatorResult<CompletableFuture<Void>, U> result;
+
+        /**
+         * Constructor.
+         *
+         * @param name  The operation name.
+         * @param tp    The topic partition that the operation is applied to.
+         * @param op    The operation.
+         */
+        GenericGroupEvent(
+            String name,
+            TopicPartition tp,
+            GenericGroupOperation<U> op
+        ) {
+            this.tp = tp;
+            this.name = name;
+            this.op = op;
+        }
+
+        /**
+         * @return The key used by the CoordinatorEventProcessor to ensure
+         * that events with the same key are not processed concurrently.
+         */
+        @Override
+        public TopicPartition key() {
+            return tp;
+        }
+
+        /**
+         * Called by the CoordinatorEventProcessor when the event is executed.
+         */
+        @Override
+        public void run() {
+            try {
+                // Get the context of the coordinator or fail if the coordinator is not in active state.
+                CoordinatorContext context = activeContextOrThrow(tp);
+
+                result = op.generateRecordsAndAppendFuture((ReplicatedGroupCoordinator) context.coordinator);

Review Comment:
   One issue here is that if `generateRecordsAndAppendFuture` thrown an exception (e.g. due to a bug), the request will never be completed because the future is not available yet. If we reuse `CoordinatorWriteEvent`, we could subscribe to the future returned by it and complete the response when an exception is raised.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +295,21 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isValidGroupId(request.groupId(), ApiKeys.forId(request.apiKey()))) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleGenericGroupOperation("generic-group-join",
+            topicPartitionFor(request.groupId()),
+            coordinator -> coordinator.genericGroupJoin(context, request, responseFuture));
+
+        return responseFuture;

Review Comment:
   We probably need to convert some of the exceptions like I did for the consumer group heartbeat request.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1072,1338 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+            // TODO: this needs to be checked in conjunction with empty group offsets.
+//            if (groups.containsKey(groupId)) {
+//                throw new IllegalStateException("Unexpected unload of active group " + groupId +
+//                    "while loading partition " + topicPartition);
+//            }
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<CompletableFuture<Void>, Record> result = EMPTY_RESULT;
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            String joinReason = request.reason();
+            if (joinReason == null || joinReason.isEmpty()) {
+                joinReason = "not provided";
+            }
+
+            if (!acceptJoiningMember(group, memberId)) {
+                group.remove(memberId);
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(UNKNOWN_MEMBER_ID)
+                    .setErrorCode(Errors.GROUP_MAX_SIZE_REACHED.code())
+                );
+
+            } else if (isUnknownMember) {
+                result = genericGroupJoinNewMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                result = genericGroupJoinExistingMember(
+                    context,
+                    request,
+                    group,
+                    joinReason,
+                    responseFuture
+                );
+            }
+
+            // Attempt to complete join group phase. We do not complete
+            // the join group phase if this is the initial rebalance.
+            if (group.isInState(PREPARING_REBALANCE) &&
+                group.hasAllMembersJoined() &&
+                group.generationId() != 0
+            ) {
+                // The only two cases where we produce records to append are when:
+                //     1) A new static member replaces an existing member during Stable state.
+                //     2) The group already completed the join phase and transitioned to Empty state.
+                // Therefore, we should not be in PreparingRebalance state.
+                if (result != EMPTY_RESULT) {
+                    throw new IllegalStateException("There are records to append but we are attempting to" +
+                        "complete the join phase.");
+                }
+
+                result = completeGenericGroupJoin(group);
+            }
+        }
+
+        return result;
+    }
+
+    /**
+     * Handle a new member generic group join.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+
+        if (group.isInState(DEAD)) {
+            // if the group is marked as dead, it means some other thread has just removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(UNKNOWN_MEMBER_ID)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else {
+            Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+            String newMemberId = group.generateMemberId(context.clientId(), groupInstanceId);
+
+            if (groupInstanceId.isPresent()) {
+                return genericGroupJoinNewStaticMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            } else {
+                return genericGroupJoinNewDynamicMember(
+                    context,
+                    request,
+                    group,
+                    newMemberId,
+                    joinReason,
+                    responseFuture
+                );
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle new static member join. If there was an existing member id for the group instance id,
+     * replace that member. Otherwise, add the member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewStaticMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String groupInstanceId = request.groupInstanceId();
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId != null) {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Replacing previously mapped member {} with this groupInstanceId.",
+                groupInstanceId, group.groupId(), group.currentState(), existingMemberId);
+
+            return updateStaticMemberAndRebalance(
+                context,
+                request,
+                group,
+                existingMemberId,
+                newMemberId,
+                joinReason,
+                responseFuture
+            );
+
+        } else {
+            log.info("Static member with groupInstanceId={} and unknown member id joins " +
+                    "group {} in {} state. Created a new member id {} for this member and added to the group.",
+                groupInstanceId, group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+    }
+
+    /**
+     * Handle a new dynamic member join. If the member id field is required, the group metadata manager
+     * will add the new member id to the pending members and respond with MEMBER_ID_REQUIRED along with
+     * the new member id for the client to join with.
+     *
+     * Otherwise, add the new member to the group and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param newMemberId     The newly generated member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinNewDynamicMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        boolean requireKnownMemberId = context.apiVersion() >= 4;
+
+        if (requireKnownMemberId) {
+            // If member id required, register the member in the pending member list and send
+            // back a response to call for another join group request with allocated member id.
+            log.info("Dynamic member with unknown member id joins group {} in {} state. " +
+                    "Created a new member id {} and requesting the member to rejoin with this id.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            group.addPendingMember(newMemberId);
+            String heartbeatKey = heartbeatKey(group.groupId(), newMemberId);
+
+            timer.schedule(
+                heartbeatKey,
+                request.sessionTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> expireGenericGroupMemberHeartbeat(group, newMemberId)
+            );
+
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(newMemberId)
+                .setErrorCode(Errors.MEMBER_ID_REQUIRED.code())
+            );
+
+        } else {
+            log.info("Dynamic member with unknown member id joins group {} in state {}. " +
+                    "Created a new member id {} and added the member to the group.",
+                group.groupId(), group.currentState(), newMemberId);
+
+            return addMemberAndRebalance(context, request, group, newMemberId, joinReason, responseFuture);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Handle a join group request for an existing member.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> genericGroupJoinExistingMember(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String memberId = request.memberId();
+        String groupInstanceId = request.groupInstanceId();
+
+        if (group.isInState(DEAD)) {
+            // If the group is marked as dead, it means the group was recently removed the group
+            // from the coordinator metadata; it is likely that the group has migrated to some other
+            // coordinator OR the group is in a transient unstable phase. Let the member retry
+            // finding the correct coordinator and rejoin.
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.COORDINATOR_NOT_AVAILABLE.code())
+            );
+        } else if (!group.supportsProtocols(
+            request.protocolType(),
+            GenericGroupMember.plainProtocolSet(request.protocols()))
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INCONSISTENT_GROUP_PROTOCOL.code())
+            );
+        } else if (group.isPendingMember(memberId)) {
+            // A rejoining pending member will be accepted. Note that pending member cannot be a static member.
+            if (groupInstanceId != null) {
+                throw new IllegalStateException("Received unexpected JoinGroup with groupInstanceId=" +
+                    groupInstanceId + " for pending member with memberId=" + memberId);
+            }
+
+            log.debug("Pending dynamic member with id {} joins group {} in {} state. Adding to the group now.",
+                memberId, group.groupId(), group.currentState());
+
+            return addMemberAndRebalance(
+                context,
+                request,
+                group,
+                memberId,
+                joinReason,
+                responseFuture
+            );
+        } else {
+            Optional<Errors> memberError = validateExistingMember(
+                group,
+                memberId,
+                groupInstanceId,
+                "join-group"
+            );
+
+            if (memberError.isPresent()) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(memberError.get().code())
+                );
+            } else {
+                GenericGroupMember member = group.member(memberId);
+                if (group.isInState(PREPARING_REBALANCE)) {
+                    return updateMemberAndRebalance(
+                        request,
+                        group,
+                        member,
+                        "Member " + member.memberId() + " is joining group during " +
+                            group.stateAsString() + "; client reason: " + joinReason,
+                        responseFuture
+                    );
+                } else if (group.isInState(COMPLETING_REBALANCE)) {
+                    if (member.matches(request.protocols())) {
+                        // Member is joining with the same metadata (which could be because it failed to
+                        // receive the initial JoinGroup response), so just return current group information
+                        // for the current generation.
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (group.isLeader(memberId)) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    } else {
+                        // Member has changed metadata, so force a rebalance
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    }
+                } else if (group.isInState(STABLE)) {
+                    if (group.isLeader(memberId)) {
+                        // Force a rebalance if the leader sends JoinGroup;
+                        // This allows the leader to trigger rebalances for changes affecting assignment
+                        // which do not affect the member metadata (such as topic metadata changes for the consumer)
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Leader " + memberId + " re-joining group during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else if (!member.matches(request.protocols())) {
+                        return updateMemberAndRebalance(
+                            request,
+                            group,
+                            member,
+                            "Updating metadata for member " + memberId + " during " +
+                                group.stateAsString() + "; client reason: " + joinReason,
+                            responseFuture
+                        );
+                    } else {
+                        // For followers with no actual change to their metadata, just return group information
+                        // for the current generation which will allow them to issue SyncGroup.
+                        responseFuture.complete(new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(memberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                } else {
+                    // Group reached unexpected (Empty) state. Let the joining member reset their generation and rejoin.
+                    log.warn("Attempt to add rejoining member {} of group {} in unexpected group state {}",
+                        memberId, group.groupId(), group.stateAsString());
+
+                    responseFuture.complete(new JoinGroupResponseData()
+                        .setMemberId(memberId)
+                        .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+                    );
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Complete the join group phase. Remove all dynamic members that have not rejoined
+     * during this stage and proceed with the next generation for this group. The generation id
+     * is incremented and the group transitions to CompletingRebalance state if there is at least
+     * one member.
+     *
+     * If the group is in Empty state, append a new group metadata record to the log. Otherwise,
+     * complete all members' join group response futures and wait for sync requests from members.
+     *
+     * @param group The group that is completing the join group phase.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> completeGenericGroupJoin(
+        GenericGroup group
+    ) {
+        timer.cancel(joinKey(group.groupId()));
+        String groupId = group.groupId();
+
+        Map<String, GenericGroupMember> notYetRejoinedDynamicMembers =
+            group.notYetRejoinedMembers().entrySet().stream()
+                .filter(entry -> !entry.getValue().isStaticMember())
+                .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+        if (!notYetRejoinedDynamicMembers.isEmpty()) {
+            notYetRejoinedDynamicMembers.values().forEach(failedMember -> {
+                group.remove(failedMember.memberId());
+                timer.cancel(heartbeatKey(group.groupId(), failedMember.memberId()));
+            });
+
+            log.info("Group {} removed dynamic members who haven't joined: {}",
+                groupId, notYetRejoinedDynamicMembers.keySet());
+        }
+
+        if (group.isInState(DEAD)) {
+            log.info("Group {} is dead, skipping rebalance stage.", groupId);
+        } else if (!group.maybeElectNewJoinedLeader() && !group.allMembers().isEmpty()) {
+            // If all members are not rejoining, we will postpone the completion
+            // of rebalance preparing stage, and send out another delayed operation
+            // until session timeout removes all the non-responsive members.
+            log.error("Group {} could not complete rebalance because no members rejoined.", groupId);
+
+            timer.schedule(
+                joinKey(groupId),
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+
+            return EMPTY_RESULT;
+
+        } else {
+            group.initNextGeneration();
+            if (group.isInState(EMPTY)) {
+                log.info("Group {} with generation {} is now empty ({}-{})",
+                    groupId, group.generationId(), topicPartition.topic(), topicPartition.partition());
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. If the broker fails before another rebalance,
+                        // the previous generation written to the log will become active again (and most likely timeout).
+                        // This should be safe since there are no active members in an empty generation, so we just warn.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                log.info("Stabilized group {} generation {} ({}-{}) with {} members",
+                    groupId, group.generationId(), topicPartition.topic(),
+                    topicPartition.partition(), group.size());
+
+                // Complete the awaiting join group response future for all the members after rebalancing
+                group.allMembers().forEach(member -> {
+
+                    List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                    if (group.isLeader(member.memberId())) {
+                        members = group.currentGenericGroupMembers();
+                    }
+
+                    JoinGroupResponseData response = new JoinGroupResponseData()
+                        .setMembers(members)
+                        .setMemberId(member.memberId())
+                        .setGenerationId(group.generationId())
+                        .setProtocolName(group.protocolName().orElse(null))
+                        .setProtocolType(group.protocolType().orElse(null))
+                        .setLeader(group.leaderOrNull())
+                        .setSkipAssignment(false)
+                        .setErrorCode(Errors.NONE.code());
+
+                    group.completeJoinFuture(member, response);
+                    rescheduleGenericGroupMemberHeartbeat(group, member, group.rebalanceTimeoutMs());
+                    member.setIsNew(false);
+
+                    group.addPendingSyncMember(member.memberId());
+                });
+
+                schedulePendingSync(group);
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Wait for sync requests for the group.
+     *
+     * @param group The group.
+     */
+    private void schedulePendingSync(GenericGroup group) {
+        timer.schedule(syncKey(group.groupId()),
+            group.rebalanceTimeoutMs(),
+            TimeUnit.MILLISECONDS,
+            () -> expirePendingSync(group, group.generationId()));
+    }
+
+    /**
+     * Invoked when the heartbeat operation is expired from the timer. Possibly remove the member and
+     * try complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expireGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        String memberId
+    ) {
+        if (group.isInState(DEAD)) {
+            log.info("Received notification of heartbeat expiration for member {} after group {} " +
+                    "had already been unloaded or deleted.",
+                memberId, group.groupId());
+
+        } else if (group.isPendingMember(memberId)) {
+            log.info("Pending member {} in group {} has been removed after session timeout expiration.",
+                memberId, group.groupId());
+
+            return removePendingMemberAndUpdateGenericGroup(group, memberId);
+        } else if (!group.hasMemberId(memberId)) {
+            log.debug("Member {} has already been removed from the group.", memberId);
+        } else {
+            GenericGroupMember member = group.member(memberId);
+            if (!member.hasSatisfiedHeartbeat()) {
+                log.info("Member {} in group {} has failed, removing it from the group.",
+                    member.memberId(), group.groupId());
+
+                return removeMemberAndUpdateGenericGroup(
+                    group,
+                    member,
+                    "removing member " + member.memberId() + " on heartbeat expiration."
+                );
+            }
+        }
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Invoked when the heartbeat key is expired from the timer. Possibly remove the member
+     * from the group and try to complete the join phase.
+     *
+     * @param group     The group.
+     * @param member    The member.
+     * @param reason    The reason for removing the member.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removeMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        GenericGroupMember member,
+        String reason
+    ) {
+        // New members may timeout with a pending JoinGroup while the group is still rebalancing, so we have
+        // to invoke the response future before removing the member. We return UNKNOWN_MEMBER_ID so
+        // that the consumer will retry the JoinGroup request if it is still active.
+        group.completeJoinFuture(member, new JoinGroupResponseData()
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setErrorCode(Errors.UNKNOWN_MEMBER_ID.code())
+        );
+        group.remove(member.memberId());
+
+        if (group.isInState(STABLE) || group.isInState(COMPLETING_REBALANCE)) {
+            return maybePrepareRebalance(group, reason);
+        } else if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        } // Do nothing if state is DEAD or EMPTY.
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Remove a pending member from the group and possibly complete the join phase.
+     *
+     * @param group     The group.
+     * @param memberId  The member id.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> removePendingMemberAndUpdateGenericGroup(
+        GenericGroup group,
+        String memberId
+    ) {
+        group.remove(memberId);
+
+        if (group.isInState(PREPARING_REBALANCE) && group.hasAllMembersJoined()) {
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Update an existing member and rebalance.
+     *
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param member          The member.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateMemberAndRebalance(
+        JoinGroupRequestData request,
+        GenericGroup group,
+        GenericGroupMember member,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        return maybePrepareRebalance(group, joinReason);
+    }
+
+    /**
+     * We are validating two things:
+     *     1. If `groupInstanceId` is present, then it exists and is mapped to `memberId`
+     *     2. The `memberId` exists in the group
+     *
+     * @param group            The generic group.
+     * @param memberId         The member id.
+     * @param groupInstanceId  The group instance id.
+     * @param operation        The API operation.
+     *
+     * @return the error.
+     */
+    private Optional<Errors> validateExistingMember(
+        GenericGroup group,
+        String memberId,
+        String groupInstanceId,
+        String operation
+    ) {
+        if (groupInstanceId == null) {
+            if (!group.hasMemberId(memberId)) {
+                return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+            } else {
+                return Optional.empty();
+            }
+        }
+
+        String existingMemberId = group.staticMemberId(groupInstanceId);
+        if (existingMemberId == null) {
+            return Optional.of(Errors.UNKNOWN_MEMBER_ID);
+        }
+
+        if (!existingMemberId.equals(memberId)) {
+            log.info("Request memberId={} for static member with groupInstanceId={} " +
+                    "is fenced by existing memberId={} during operation {}",
+                memberId, groupInstanceId, existingMemberId, operation);
+            
+            return Optional.of(Errors.FENCED_INSTANCE_ID);
+        }
+        
+        return Optional.empty();
+    }
+
+    /**
+     * Add a member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group to add the member.
+     * @param memberId        The member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> addMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String memberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        Optional<String> groupInstanceId = Optional.ofNullable(request.groupInstanceId());
+        GenericGroupMember member = new GenericGroupMember(
+            memberId,
+            groupInstanceId,
+            context.clientId(),
+            context.clientAddress().toString(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            request.protocolType(),
+            request.protocols()
+        );
+
+        member.setIsNew(true);
+
+        // Update the newMemberAdded flag to indicate that the initial rebalance can be further delayed
+        if (group.isInState(PREPARING_REBALANCE) && group.generationId() == 0) {
+            group.setNewMemberAdded(true);
+        }
+        
+        group.add(member, responseFuture);
+
+        // The session timeout does not affect new members since they do not have their memberId and
+        // cannot send heartbeats. Furthermore, we cannot detect disconnects because sockets are muted
+        // while the JoinGroup request is parked. If the client does disconnect (e.g. because of a request
+        // timeout during a long rebalance), they may simply retry which will lead to a lot of defunct
+        // members in the rebalance. To prevent this going on indefinitely, we time out JoinGroup requests
+        // for new members. If the new member is still there, we expect it to retry.
+        rescheduleGenericGroupMemberHeartbeat(group, member, genericGroupNewMemberJoinTimeoutMs);
+
+        return maybePrepareRebalance(group, "Adding new member " + memberId + " with group instance id " +
+            request.groupInstanceId() + "; client reason: " + joinReason);
+    }
+
+    /**
+     * Prepare a rebalance if the group is in a valid state.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> maybePrepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        if (group.canRebalance()) {
+            return prepareRebalance(group, reason);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Prepare a rebalance.
+     *
+     * @param group           The group to rebalance.
+     * @param reason          The reason for the rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    CoordinatorResult<CompletableFuture<Void>, Record> prepareRebalance(
+        GenericGroup group,
+        String reason
+    ) {
+        // If any members are awaiting sync, cancel their request and have them rejoin.
+        if (group.isInState(COMPLETING_REBALANCE)) {
+            resetAndPropagateAssignmentWithError(group, Errors.REBALANCE_IN_PROGRESS);
+        }
+
+        // If a sync expiration is pending, cancel it.
+        removeSyncExpiration(group);
+
+        boolean isInitialRebalance = group.generationId() == 0;
+        if (isInitialRebalance) {
+            // The group is new. Provide more time for the members to join.
+            int delayMs = genericGroupInitialRebalanceDelayMs;
+            int remainingMs = Math.max(group.rebalanceTimeoutMs() - genericGroupInitialRebalanceDelayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                delayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, delayMs, remainingMs)
+            );
+        }
+
+
+        group.transitionTo(PREPARING_REBALANCE);
+
+        log.info("Preparing to rebalance group {} in state {} with old generation {} ({}-{}) (reason: {})",
+            group.groupId(), group.currentState(), group.generationId(),
+            topicPartition.topic(), topicPartition.partition(), reason);
+
+        return isInitialRebalance ? EMPTY_RESULT : tryCompleteJoinElseSchedule(group);
+    }
+
+    /**
+     * Try to complete the join phase. Otherwise, schedule a new join operation.
+     *
+     * @param group The group.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteJoinElseSchedule(
+        GenericGroup group
+    ) {
+        String joinKey = joinKey(group.groupId());
+        if (group.hasAllMembersJoined()) {
+            // All members have joined. Proceed to sync phase.
+            return completeGenericGroupJoin(group);
+        } else {
+            timer.schedule(
+                joinKey,
+                group.rebalanceTimeoutMs(),
+                TimeUnit.MILLISECONDS,
+                () -> completeGenericGroupJoin(group)
+            );
+            return EMPTY_RESULT;
+        }
+    }
+
+    /**
+     * Try to complete the join phase of the initial rebalance.
+     * Otherwise, extend the rebalance.
+     *
+     * @param group The group under initial rebalance.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> tryCompleteInitialRebalanceElseSchedule(
+        GenericGroup group,
+        int delayMs,
+        int remainingMs
+    ) {
+        if (group.newMemberAdded() && remainingMs != 0) {
+            // A new member was added. Extend the delay.
+            group.setNewMemberAdded(false);
+            int newDelayMs = Math.min(genericGroupInitialRebalanceDelayMs, remainingMs);
+            int newRemainingMs = Math.max(remainingMs - delayMs, 0);
+
+            timer.schedule(
+                joinKey(group.groupId()),
+                newDelayMs,
+                TimeUnit.MILLISECONDS,
+                () -> tryCompleteInitialRebalanceElseSchedule(group, newDelayMs, newRemainingMs)
+            );
+        } else {
+            // No more time remaining. Complete the join phase.
+            return completeGenericGroupJoin(group);
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Reset assignment for all members and propagate the error to all members in the group.
+     * 
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void resetAndPropagateAssignmentWithError(GenericGroup group, Errors error) {
+        if (!group.isInState(COMPLETING_REBALANCE)) {
+            throw new IllegalStateException("Group " + group.groupId() + " must be in " + COMPLETING_REBALANCE.name() +
+                " state but is in " + group.currentState() + ".");
+        }
+
+        group.allMembers().forEach(member -> member.setAssignment(GenericGroupMember.EMPTY_ASSIGNMENT));
+        propagateAssignment(group, error);
+    }
+
+    /**
+     * Propagate assignment and error to all members.
+     *
+     * @param group  The group.
+     * @param error  The error to propagate.
+     */
+    private void propagateAssignment(GenericGroup group, Errors error) {
+        Optional<String> protocolName = Optional.empty();
+        Optional<String> protocolType = Optional.empty();
+        if (error == Errors.NONE) {
+            protocolName = group.protocolName();
+            protocolType = group.protocolType();
+        }
+
+        for (GenericGroupMember member : group.allMembers()) {
+            if ((member.assignment() == null || member.assignment().length == 0) && error == Errors.NONE) {
+                log.warn("Sending empty assignment to member {} of {} for " + "generation {} with no errors",
+                    member.memberId(), group.groupId(), group.generationId());
+            }
+
+            if (group.completeSyncFuture(member,
+                new SyncGroupResponseData()
+                    .setProtocolName(protocolName.orElse(null))
+                    .setProtocolType(protocolType.orElse(null))
+                    .setAssignment(member.assignment())
+                    .setErrorCode(error.code()))) {
+
+                // Reset the session timeout for members after propagating the member's assignment.
+                // This is because if any member's session expired while we were still awaiting either
+                // the leader sync group or the append future, its expiration will be ignored and no
+                // future heartbeat expectations will not be scheduled.
+                rescheduleGenericGroupMemberHeartbeat(group, member);
+            }
+        }
+    }
+
+    /**
+     * Complete and schedule next heartbeat.
+     *
+     * @param group    The group.
+     * @param member   The member.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member
+    ) {
+        rescheduleGenericGroupMemberHeartbeat(group, member, member.sessionTimeoutMs());
+    }
+
+    /**
+     * Reschedule the heartbeat.
+     *
+     * @param group      The group.
+     * @param member     The member.
+     * @param timeoutMs  The timeout for the new heartbeat.
+     */
+    private void rescheduleGenericGroupMemberHeartbeat(
+        GenericGroup group,
+        GenericGroupMember member,
+        long timeoutMs
+    ) {
+        String heartbeatKey = heartbeatKey(group.groupId(), member.memberId());
+
+        // Reschedule the next heartbeat expiration deadline
+        timer.schedule(heartbeatKey,
+            timeoutMs,
+            TimeUnit.MILLISECONDS,
+            () -> expireGenericGroupMemberHeartbeat(group, member.memberId()));
+    }
+
+    /**
+     * Remove the sync key from the timer and clear all pending sync members from the group.
+     * Invoked when a new rebalance is triggered.
+     *
+     * @param group  The group.
+     */
+    private void removeSyncExpiration(GenericGroup group) {
+        group.clearPendingSyncMembers();
+        timer.cancel(syncKey(group.groupId()));
+    }
+
+    /**
+     * Expire pending sync.
+     *
+     * @param group           The group.
+     * @param generationId    The generation when the pending sync was originally scheduled.
+     *
+     * @return The coordinator result that will be appended to the log.
+     * */
+    private CoordinatorResult<CompletableFuture<Void>, Record> expirePendingSync(
+        GenericGroup group,
+        int generationId
+    ) {
+        if (generationId != group.generationId()) {
+            log.error("Received unexpected notification of sync expiration for {} with an old " +
+                "generation {} while the group has {}.", group.groupId(), generationId, group.generationId());
+        } else {
+            if (group.isInState(DEAD) || group.isInState(EMPTY) || group.isInState(PREPARING_REBALANCE)) {
+                log.error("Received unexpected notification of sync expiration after group {} already " +
+                    "transitioned to {} state.", group.groupId(), group.stateAsString());
+            } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+                if (!group.hasReceivedSyncFromAllMembers()) {
+                    Set<String> pendingSyncMembers = group.allPendingSyncMembers();
+                    pendingSyncMembers.forEach(memberId -> {
+                        group.remove(memberId);
+                        timer.cancel(heartbeatKey(group.groupId(), memberId));
+                    });
+
+                    log.debug("Group {} removed members who haven't sent their sync requests: {}",
+                        group.groupId(), pendingSyncMembers);
+
+                    return prepareRebalance(group, "Removing " + pendingSyncMembers + " on pending sync request expiration");
+                }
+            }
+        }
+
+        return EMPTY_RESULT;
+    }
+
+    /**
+     * Checks whether the group can accept a joining member.
+     *
+     * @param group      The group.
+     * @param memberId   The member.
+     *
+     * @return whether the group can accept a joining member.
+     */
+    private boolean acceptJoiningMember(GenericGroup group, String memberId) {
+        if (group.isInState(EMPTY) || group.isInState(DEAD)) {
+            // Always accept the request when the group is empty or dead
+            return true;
+
+        } else if (group.isInState(PREPARING_REBALANCE)) {
+            // An existing member is accepted if it is already awaiting. New members are accepted
+            // up to the max group size. Note that the number of awaiting members is used here
+            // for two reasons:
+            // 1) the group size is not reliable as it could already be above the max group size
+            //    if the max group size was reduced.
+            // 2) using the number of awaiting members allows to kick out the last rejoining
+            //    members of the group.
+            return (group.hasMemberId(memberId) && group.member(memberId).isAwaitingJoin()) ||
+                group.numAwaitingJoinResponse() < genericGroupMaxSize;
+
+        } else if (group.isInState(COMPLETING_REBALANCE) || group.isInState(STABLE)) {
+            // An existing member is accepted. New members are accepted up to the max group size.
+            // Note that the group size is used here. When the group transitions to CompletingRebalance,
+            // members who haven't rejoined are removed.
+            return group.hasMemberId(memberId) || group.size() < genericGroupMaxSize;
+        } else {
+            throw new IllegalStateException("Cannot handle state " + group.stateAsString());
+        }
+    }
+
+    /**
+     * Update a static member and rebalance.
+     *
+     * @param context         The request context.
+     * @param request         The join group request.
+     * @param group           The group of the static member.
+     * @param oldMemberId     The existing static member id.
+     * @param newMemberId     The new joining static member id.
+     * @param joinReason      The client reason for the join request.
+     * @param responseFuture  The response future to complete.
+     *
+     * @return The coordinator result that will be appended to the log.
+     */
+    private CoordinatorResult<CompletableFuture<Void>, Record> updateStaticMemberAndRebalance(
+        RequestContext context,
+        JoinGroupRequestData request,
+        GenericGroup group,
+        String oldMemberId,
+        String newMemberId,
+        String joinReason,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        String currentLeader = group.leaderOrNull();
+        GenericGroupMember member = group.replaceStaticMember(request.groupInstanceId(), oldMemberId, newMemberId);
+
+        // Heartbeat of old member id will expire without effect since the group no longer contains that member id.
+        // New heartbeat shall be scheduled with new member id.
+        rescheduleGenericGroupMemberHeartbeat(group, member);
+
+        int oldRebalanceTimeoutMs = member.rebalanceTimeoutMs();
+        int oldSessionTimeoutMs = member.sessionTimeoutMs();
+        JoinGroupRequestProtocolCollection oldProtocols = member.supportedProtocols();
+
+        group.updateMember(
+            member,
+            request.protocols(),
+            request.rebalanceTimeoutMs(),
+            request.sessionTimeoutMs(),
+            responseFuture
+        );
+
+        if (group.isInState(STABLE)) {
+            // Check if group's selected protocol of next generation will change, if not, simply store group to persist
+            // the updated static member, if yes, rebalance should be triggered to keep the group's assignment
+            // and selected protocol consistent
+            String groupInstanceId = request.groupInstanceId();
+            String selectedProtocolForNextGeneration = group.selectProtocol();
+            if (group.protocolName().orElse("").equals(selectedProtocolForNextGeneration)) {
+                log.info("Static member which joins during Stable stage and doesn't affect " +
+                    "the selected protocol will not trigger a rebalance.");
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    // Starting from version 9 of the JoinGroup API, static members are able to
+                    // skip running the assignor based on the `SkipAssignment` field. We leverage
+                    // this to tell the leader that it is the leader of the group but by skipping
+                    // running the assignor while the group is in stable state.
+                    // Notes:
+                    // 1) This allows the leader to continue monitoring metadata changes for the
+                    // group. Note that any metadata changes happening while the static leader is
+                    // down won't be noticed.
+                    // 2) The assignors are not idempotent nor free from side effects. This is why
+                    // we skip entirely the assignment step as it could generate a different group
+                    // assignment which would be ignored by the group coordinator because the group
+                    // is the stable state.
+                    boolean supportSkippingAssignment = context.apiVersion() >= 9;
+
+                    if (t != null) {
+                        log.warn("Failed to persist metadata for group {}: {}", group.groupId(), t.getMessage());
+
+                        // Failed to persist the member id of the given static member, revert the update of the static member in the group.
+                        group.updateMember(member, oldProtocols, oldRebalanceTimeoutMs, oldSessionTimeoutMs, null);
+                        GenericGroupMember oldMember = group.replaceStaticMember(groupInstanceId, newMemberId, oldMemberId);
+                        rescheduleGenericGroupMemberHeartbeat(group, oldMember);
+
+                        short errorCode = Errors.forException(t).code();
+                        responseFuture.complete(
+                            new JoinGroupResponseData()
+                                .setMembers(Collections.emptyList())
+                                .setMemberId(UNKNOWN_MEMBER_ID)
+                                .setGenerationId(group.generationId())
+                                .setProtocolName(group.protocolName().orElse(null))
+                                .setProtocolType(group.protocolType().orElse(null))
+                                .setLeader(currentLeader)
+                                .setSkipAssignment(false)
+                                .setErrorCode(errorCode)
+                        );
+
+                    } else if (supportSkippingAssignment) {
+                        boolean isLeader = group.isLeader(newMemberId);
+                        List<JoinGroupResponseData.JoinGroupResponseMember> members = Collections.emptyList();
+                        if (isLeader) {
+                            members = group.currentGenericGroupMembers();
+                        }
+                        group.completeJoinFuture(member, new JoinGroupResponseData()
+                            .setMembers(members)
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(group.leaderOrNull())
+                            .setSkipAssignment(isLeader)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+
+                    } else {
+                        // Prior to version 9 of the JoinGroup API, we wanted to avoid current leader
+                        // performing trivial assignment while the group is in stable stage, because
+                        // the new assignment in leader's next sync call won't be broadcast by a stable group.
+                        // This could be guaranteed by always returning the old leader id so that the current
+                        // leader won't assume itself as a leader based on the returned message, since the new
+                        // member.id won't match returned leader id, therefore no assignment will be performed.
+
+                        group.completeJoinFuture(member, new JoinGroupResponseData()
+                            .setMembers(Collections.emptyList())
+                            .setMemberId(newMemberId)
+                            .setGenerationId(group.generationId())
+                            .setProtocolName(group.protocolName().orElse(null))
+                            .setProtocolType(group.protocolType().orElse(null))
+                            .setLeader(currentLeader)
+                            .setSkipAssignment(false)
+                            .setErrorCode(Errors.NONE.code())
+                        );
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                return new CoordinatorResult<>(records, appendFuture);
+
+            } else {
+                return maybePrepareRebalance(
+                    group,
+                    "Group's selectedProtocol will change because static member " +
+                        member.memberId() + " with instance id " + groupInstanceId +
+                        " joined with change of protocol; " + "client reason: " + joinReason
+                );
+            }
+
+        } else if (group.isInState(COMPLETING_REBALANCE)) {
+            // if the group is in after-sync stage, upon getting a new join-group of a known static member
+            // we should still trigger a new rebalance, since the old member may already be sent to the leader
+            // for assignment, and hence when the assignment gets back there would be a mismatch of the old member id
+            // with the new replaced member id. As a result the new member id would not get any assignment.
+            return prepareRebalance(group,
+                "Updating metadata for static member " + member.memberId() + " with instance id " +
+                    request.groupInstanceId() + "; client reason: " + joinReason);
+
+        } else if (group.isInState(EMPTY) || group.isInState(DEAD)) {
+            throw new IllegalStateException("Group " + group.groupId() + " was not supposed to be in the state " +
+                group.stateAsString() + " when the unknown static member " + request.groupInstanceId() + " rejoins.");

Review Comment:
   i expected that KafkaApis#handleJoinGroupRequest will handle them. is that not the 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] jeffkbkim commented on a diff in pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1362 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            if (group.isNew()) {
+                // If a group was newly created, we need to append records to the log
+                // to commit the group to the timeline datastructure.
+
+                CompletableFuture<Void> appendFuture = new CompletableFuture<>();
+                appendFuture.whenComplete((__, t) -> {
+                    if (t != null) {
+                        // We failed to write the empty group metadata. This will revert the snapshot, removing
+                        // the newly created group.
+                        log.warn("Failed to write empty metadata for group {}: {}", group.groupId(), t.getMessage());
+                    }
+                });
+
+                List<Record> records = Collections.singletonList(
+                    RecordHelpers.newGroupMetadataRecord(group, metadataImage.features().metadataVersion())
+                );
+
+                result = new CoordinatorResult<>(records, appendFuture);
+                genericGroupJoinMember(context, request, group, isUnknownMember, responseFuture);

Review Comment:
   the approach is hacky. will be thinking of a different approach to resolve 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 pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

Posted by "jeffkbkim (via GitHub)" <gi...@apache.org>.
jeffkbkim commented on PR #13870:
URL: https://github.com/apache/kafka/pull/13870#issuecomment-1625762579

   @dajac @CalvinConfluent 
   
   thanks for the review, i have addressed your 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -874,4 +1087,1348 @@ public void replay(
             consumerGroup.updateMember(newMember);
         }
     }
+
+    /**
+     * Replays GroupMetadataKey/Value to update the soft state of
+     * the generic group.
+     *
+     * @param key   A GroupMetadataKey key.
+     * @param value A GroupMetadataValue record.
+     */
+    public void replay(
+        GroupMetadataKey key,
+        GroupMetadataValue value,
+        short version
+    ) {
+        String groupId = key.group();
+
+        if (value == null)  {
+            // Tombstone. Group should not be added.
+        } else {
+            List<GenericGroupMember> loadedMembers = new ArrayList<>();
+            for (GroupMetadataValue.MemberMetadata member : value.members()) {
+                int rebalanceTimeout = version == 0 ? member.sessionTimeout() : member.rebalanceTimeout();
+
+                JoinGroupRequestProtocolCollection supportedProtocols = new JoinGroupRequestProtocolCollection();
+                supportedProtocols.add(new JoinGroupRequestProtocol()
+                    .setName(value.protocol())
+                    .setMetadata(member.subscription()));
+
+                GenericGroupMember loadedMember = new GenericGroupMember(
+                    member.memberId(),
+                    Optional.ofNullable(member.groupInstanceId()),
+                    member.clientId(),
+                    member.clientHost(),
+                    rebalanceTimeout,
+                    member.sessionTimeout(),
+                    value.protocolType(),
+                    supportedProtocols,
+                    member.assignment()
+                );
+
+                loadedMembers.add(loadedMember);
+            }
+
+            String protocolType = value.protocolType();
+
+            GenericGroup genericGroup = new GenericGroup(
+                this.logContext,
+                groupId,
+                loadedMembers.isEmpty() ? EMPTY : STABLE,
+                time,
+                value.generation(),
+                protocolType == null || protocolType.isEmpty() ? Optional.empty() : Optional.of(protocolType),
+                Optional.ofNullable(value.protocol()),
+                Optional.ofNullable(value.leader()),
+                value.currentStateTimestamp() == -1 ? Optional.empty() : Optional.of(value.currentStateTimestamp())
+            );
+
+            loadedMembers.forEach(member -> {
+                genericGroup.add(member, null);
+                log.info("Loaded member {} in group {} with generation {}.",
+                    member.memberId(), groupId, genericGroup.generationId());
+            });
+
+            genericGroup.setSubscribedTopics(
+                genericGroup.computeSubscribedTopics()
+            );
+        }
+    }
+
+    /**
+     * Handle a JoinGroupRequest.
+     *
+     * @param context The request context.
+     * @param request The actual JoinGroup request.
+     *
+     * @return The result that contains records to append if the join group phase completes.
+     */
+    public CoordinatorResult<Void, Record> genericGroupJoin(
+        RequestContext context,
+        JoinGroupRequestData request,
+        CompletableFuture<JoinGroupResponseData> responseFuture
+    ) {
+        CoordinatorResult<Void, Record> result = EMPTY_RESULT;
+
+        String groupId = request.groupId();
+        String memberId = request.memberId();
+        int sessionTimeoutMs = request.sessionTimeoutMs();
+
+        if (sessionTimeoutMs < genericGroupMinSessionTimeoutMs ||
+            sessionTimeoutMs > genericGroupMaxSessionTimeoutMs
+        ) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(memberId)
+                .setErrorCode(Errors.INVALID_SESSION_TIMEOUT.code())
+            );
+        } else {
+            boolean isUnknownMember = memberId.equals(UNKNOWN_MEMBER_ID);
+            // Group is created if it does not exist and the member id is UNKNOWN. if member
+            // is specified but group does not exist, request is rejected with GROUP_ID_NOT_FOUND
+            GenericGroup group;
+            try {
+                group = getOrMaybeCreateGenericGroup(groupId, isUnknownMember);
+            } catch (Throwable t) {
+                responseFuture.complete(new JoinGroupResponseData()
+                    .setMemberId(memberId)
+                    .setErrorCode(Errors.forException(t).code())
+                );
+                return EMPTY_RESULT;
+            }
+
+            CoordinatorResult<Void, Record> newGroupResult = EMPTY_RESULT;
+            if (group.isNew()) {

Review Comment:
   i misunderstood, moved to using a local variable



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java:
##########
@@ -150,19 +215,39 @@ GroupMetadataManager build() {
                 throw new IllegalStateException("Assignors must be set before building.");
             }
 
+            if (topicPartition == null) {
+                throw new IllegalStateException("TopicPartition must be set before building.");
+            }
+
+            if (timer == null) {
+                throw new IllegalStateException("Timer must be set before building.");
+            }
+
             return new GroupMetadataManager(
                 snapshotRegistry,
                 logContext,
-                time,
                 assignors,
                 metadataImage,
+                consumerGroupMetadataRefreshIntervalMs,
+                topicPartition,
                 consumerGroupMaxSize,
                 consumerGroupHeartbeatIntervalMs,
-                consumerGroupMetadataRefreshIntervalMs

Review Comment:
   reverted the ordering



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -266,9 +282,32 @@ public CompletableFuture<JoinGroupResponseData> joinGroup(
             return FutureUtils.failedFuture(Errors.COORDINATOR_NOT_AVAILABLE.exception());
         }
 
-        return FutureUtils.failedFuture(Errors.UNSUPPORTED_VERSION.exception(
-            "This API is not implemented yet."
-        ));
+        CompletableFuture<JoinGroupResponseData> responseFuture = new CompletableFuture<>();
+
+        if (!isGroupIdNotEmpty(request.groupId())) {
+            responseFuture.complete(new JoinGroupResponseData()
+                .setMemberId(request.memberId())
+                .setErrorCode(Errors.INVALID_GROUP_ID.code()));
+
+            return responseFuture;
+        }
+
+        runtime.scheduleWriteOperation("generic-group-join",
+            topicPartitionFor(request.groupId()),
+            coordinator -> coordinator.genericGroupJoin(context, request, responseFuture)
+        ).exceptionally(exception -> {
+            log.error("Request {} hit an unexpected exception: {}",
+                request, exception.getMessage());
+
+            if (!responseFuture.isDone()) {
+                Errors clientError = toResponseError(Errors.forException(exception));

Review Comment:
   the responseFuture if completed inside genericGroupJoin will have an error code corresponding to the join group business logic. basically, we have already completed with the appropriate error if the response future is already completed at this line.
   
   the append future error is from the append/commit process which needs to be converted if we complete the response error 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -579,4 +618,32 @@ public void shutdown() {
         Utils.closeQuietly(runtime, "coordinator runtime");
         log.info("Shutdown complete.");
     }
+
+    private static boolean isGroupIdNotEmpty(String groupId) {
+        return groupId != null && !groupId.isEmpty();
+    }
+
+    private static Errors toResponseError(Errors appendError) {
+        switch (appendError) {
+            case UNKNOWN_TOPIC_OR_PARTITION:
+            case NOT_ENOUGH_REPLICAS:
+            case NOT_ENOUGH_REPLICAS_AFTER_APPEND:
+                return COORDINATOR_NOT_AVAILABLE;
+
+            case NOT_LEADER_OR_FOLLOWER:
+            case KAFKA_STORAGE_ERROR:
+                return NOT_COORDINATOR;
+
+            case REQUEST_TIMED_OUT:

Review Comment:
   where should i look to confirm/learn 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] dajac commented on a diff in pull request #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2402,143 +2610,2090 @@ public void testOnNewMetadataImage() {
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = genericGroup(context, "group-id");
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = genericGroup(context, "group-id");
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
         }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+
+        // Send as static member join.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setGroupInstanceId("group-instance-id"), true, true, null);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(firstResponse.leader(), firstMemberId);
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, group.allMembers().iterator().next().memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+        assertNull(response.protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertFalse(responseFuture.isDone());
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+        context.timer.advanceClock(2);
+
+        // Advance clock past initial rebalance delay and verify futures are not completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+
+        // Advance clock beyond recomputed delay and make sure the futures have completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldDelayRebalanceUptoRebalanceTimeout() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs + 1);
+
+        CompletableFuture<JoinGroupResponseData> thirdMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        // Advance clock right before rebalance timeout.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+        assertFalse(thirdMemberResponseFuture.isDone());
+
+        // Advance clock beyond rebalance timeout.
+        context.timer.advanceClock(1);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertTrue(thirdMemberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), thirdMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupReplaceStaticMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .build();
+
+
+        // Send join group as static member.
+        CompletableFuture<JoinGroupResponseData> oldMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(oldMemberResponseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Replace static member with new member id. Old member id should be fenced.
+        CompletableFuture<JoinGroupResponseData> newMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        assertFalse(newMemberResponseFuture.isDone());
+        assertTrue(oldMemberResponseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Complete join for new member.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(newMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), newMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemovePendingMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(0, group.size());
+        assertEquals(1, group.numPendingJoinMembers());
+
+        // Advance clock by session timeout. Pending member should be removed from group as heartbeat expires.
+        context.timer.advanceClock(1000);
+        assertEquals(0, group.numPendingJoinMembers());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemoveMember() throws Exception {
+        // Set initial rebalance delay to simulate a long running rebalance.
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupInitialRebalanceDelayMs(10 * 60 * 1000)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        String memberId = group.leaderOrNull();
+        assertEquals(1, group.size());
+
+        // Advance clock by new member join timeout. Member should be removed from group as heartbeat expires.
+        // A group that transitions to Empty after completing join phase will generate records.
+        context.timer.expectResult(heartbeatKey("group-id", memberId), new ExpectedGenericGroupResult(
+            Collections.singletonList(newGroupMetadataRecord("group-id",
+                new GroupMetadataValue()
+                    .setMembers(Collections.emptyList())
+                    .setGeneration(1)
+                    .setLeader(null)
+                    .setProtocolType("consumer")
+                    .setProtocol(null)
+                    .setCurrentStateTimestamp(context.time.milliseconds()),
+                MetadataVersion.latest())),
+            Errors.NONE,
+            false
+        ));
+
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.size());
+    }
+
+    @Test
+    public void testExistingMemberJoinDeadGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertTrue(group.hasMemberId(memberId));
+
+        group.transitionTo(DEAD);
+
+        response = context.joinGenericGroupAsDynamicMember(request.setMemberId(memberId));
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupExistingPendingMemberWithGroupInstanceIdThrowsException() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertTrue(responseFuture.isDone());
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        assertThrows(IllegalStateException.class,
+            () -> context.sendGenericGroupJoin(request.setMemberId(memberId).setGroupInstanceId("group-instance-id"))
+        );
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        GenericGroupMember member = group.member(memberId);
+
+        assertEquals(protocols, member.supportedProtocols());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(1, group.generationId());
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        // Send updated member metadata. This should trigger a rebalance and complete the join phase.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(memberId)
+            .setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+        assertEquals(protocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingLeaderTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");

Review Comment:
   Should we move `genericGroup` to the context?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2402,143 +2610,2090 @@ public void testOnNewMetadataImage() {
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = genericGroup(context, "group-id");
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = genericGroup(context, "group-id");
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
         }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+
+        // Send as static member join.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setGroupInstanceId("group-instance-id"), true, true, null);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(firstResponse.leader(), firstMemberId);
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, group.allMembers().iterator().next().memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+        assertNull(response.protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertFalse(responseFuture.isDone());
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+        context.timer.advanceClock(2);
+
+        // Advance clock past initial rebalance delay and verify futures are not completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+
+        // Advance clock beyond recomputed delay and make sure the futures have completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldDelayRebalanceUptoRebalanceTimeout() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs + 1);
+
+        CompletableFuture<JoinGroupResponseData> thirdMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        // Advance clock right before rebalance timeout.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+        assertFalse(thirdMemberResponseFuture.isDone());
+
+        // Advance clock beyond rebalance timeout.
+        context.timer.advanceClock(1);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertTrue(thirdMemberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), thirdMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupReplaceStaticMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .build();
+

Review Comment:
   nit: Empty line.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2402,143 +2610,2090 @@ public void testOnNewMetadataImage() {
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = genericGroup(context, "group-id");
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = genericGroup(context, "group-id");
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
         }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+
+        // Send as static member join.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setGroupInstanceId("group-instance-id"), true, true, null);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(firstResponse.leader(), firstMemberId);
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, group.allMembers().iterator().next().memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+        assertNull(response.protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertFalse(responseFuture.isDone());
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+        context.timer.advanceClock(2);
+
+        // Advance clock past initial rebalance delay and verify futures are not completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+
+        // Advance clock beyond recomputed delay and make sure the futures have completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldDelayRebalanceUptoRebalanceTimeout() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs + 1);
+
+        CompletableFuture<JoinGroupResponseData> thirdMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        // Advance clock right before rebalance timeout.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+        assertFalse(thirdMemberResponseFuture.isDone());
+
+        // Advance clock beyond rebalance timeout.
+        context.timer.advanceClock(1);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertTrue(thirdMemberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), thirdMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupReplaceStaticMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .build();
+
+
+        // Send join group as static member.
+        CompletableFuture<JoinGroupResponseData> oldMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(oldMemberResponseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Replace static member with new member id. Old member id should be fenced.
+        CompletableFuture<JoinGroupResponseData> newMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        assertFalse(newMemberResponseFuture.isDone());
+        assertTrue(oldMemberResponseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Complete join for new member.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(newMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), newMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemovePendingMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(0, group.size());
+        assertEquals(1, group.numPendingJoinMembers());
+
+        // Advance clock by session timeout. Pending member should be removed from group as heartbeat expires.
+        context.timer.advanceClock(1000);
+        assertEquals(0, group.numPendingJoinMembers());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemoveMember() throws Exception {
+        // Set initial rebalance delay to simulate a long running rebalance.
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupInitialRebalanceDelayMs(10 * 60 * 1000)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        String memberId = group.leaderOrNull();
+        assertEquals(1, group.size());
+
+        // Advance clock by new member join timeout. Member should be removed from group as heartbeat expires.
+        // A group that transitions to Empty after completing join phase will generate records.
+        context.timer.expectResult(heartbeatKey("group-id", memberId), new ExpectedGenericGroupResult(
+            Collections.singletonList(newGroupMetadataRecord("group-id",
+                new GroupMetadataValue()
+                    .setMembers(Collections.emptyList())
+                    .setGeneration(1)
+                    .setLeader(null)
+                    .setProtocolType("consumer")
+                    .setProtocol(null)
+                    .setCurrentStateTimestamp(context.time.milliseconds()),
+                MetadataVersion.latest())),
+            Errors.NONE,
+            false
+        ));
+
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.size());
+    }
+
+    @Test
+    public void testExistingMemberJoinDeadGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertTrue(group.hasMemberId(memberId));
+
+        group.transitionTo(DEAD);
+
+        response = context.joinGenericGroupAsDynamicMember(request.setMemberId(memberId));
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupExistingPendingMemberWithGroupInstanceIdThrowsException() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertTrue(responseFuture.isDone());
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        assertThrows(IllegalStateException.class,
+            () -> context.sendGenericGroupJoin(request.setMemberId(memberId).setGroupInstanceId("group-instance-id"))
+        );
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        GenericGroupMember member = group.member(memberId);
+
+        assertEquals(protocols, member.supportedProtocols());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(1, group.generationId());
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        // Send updated member metadata. This should trigger a rebalance and complete the join phase.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(memberId)
+            .setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+        assertEquals(protocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingLeaderTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertTrue(group.isLeader(memberId));
+        assertEquals(1, group.generationId());
+
+        group.transitionTo(STABLE);
+        // Sending join group as leader should trigger a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with updated metadata. This should trigger a rebalance.
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        JoinGroupRequestData memberRequest = request.setMemberId(memberId).setProtocols(protocols);
+        memberResponseFuture = context.sendGenericGroupJoin(memberRequest);
+
+        assertFalse(memberResponseFuture.isDone());
+
+        // Leader rejoins. This completes the join group phase.
+        leaderResponseFuture = context.sendGenericGroupJoin(request.setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(3, group.generationId());
+        assertEquals(2, group.size());
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberDoesNotTriggerRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with no metadata changes. This does not trigger a rebalance.
+        memberResponseFuture = context.sendGenericGroupJoin(request.setMemberId(memberId));
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, memberResponseFuture.get(5, TimeUnit.SECONDS).generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberInEmptyState() throws Exception {
+        // Existing member joins a group that is in Empty/Dead state. Ask member to rejoin with generation id reset.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(-1, responseFuture.get(5, TimeUnit.SECONDS).generationId());
+    }
+
+    @Test
+    public void testCompleteJoinRemoveNotYetRejoinedDynamicMembers() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.generationId());
+
+        // Add new member. This triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout. This will expire the leader as it has not rejoined.
+        context.timer.advanceClock(1000);
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertTrue(group.hasMemberId(memberResponseFuture.get(5, TimeUnit.SECONDS).memberId()));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseInEmptyStateSkipsRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+        assertEquals(0, group.generationId());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        group.transitionTo(DEAD);
+
+        // Advance clock by initial rebalance delay to complete join phase.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertEquals(0, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseNoMembersRejoinedExtendsJoinPhase() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("first-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(30000)
+            .withRebalanceTimeoutMs(10000)
+            .build();
+
+        // First member joins group and completes join phase.
+        JoinGroupResponseData firstMemberResponse = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), firstMemberResponse.errorCode());
+        String firstMemberId = firstMemberResponse.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        // Second member joins and group goes into rebalancing state.
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setGroupInstanceId("second-instance-id"));
+
+        // First static member rejoins and completes join phase.
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        String secondMemberId = secondMemberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        // Trigger a rebalance. No members rejoined.
+        context.groupMetadataManager.prepareRebalance(group, "trigger rebalance");
+
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(0, group.numAwaitingJoinResponse());
+
+        // Advance clock by rebalance timeout to complete join phase. As long as both members have not
+        // rejoined, we extend the join phase.
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        // Let first and second member rejoin. This should complete the join phase.
+        firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(secondMemberId)
+            .setGroupInstanceId("second-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(3, group.generationId());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testReplaceStaticMemberInStableStateNoError(
+        boolean supportSkippingAssignment
+    ) throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, supportSkippingAssignment);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String oldMemberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. This should update the log with the generated member id.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request
+                .setProtocols(protocols)
+                .setRebalanceTimeoutMs(7000)
+                .setSessionTimeoutMs(4500),
+            true,
+            supportSkippingAssignment,
+            new ExpectedGenericGroupResult(Errors.NONE, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        String newMemberId = group.staticMemberId("group-instance-id");
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMemberId)
+            .setMemberId(newMemberId)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(supportSkippingAssignment)
+            .setErrorCode(Errors.NONE.code());
+
+        if (supportSkippingAssignment) {
+            expectedResponse
+                .setMembers(Collections.singletonList(
+                    new JoinGroupResponseData.JoinGroupResponseMember()
+                        .setMemberId(newMemberId)
+                        .setGroupInstanceId("group-instance-id")
+                        .setMetadata(protocols.find("range").metadata())
+                    ))
+                .setLeader(newMemberId);
+        }
+
+        GenericGroupMember updatedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        assertEquals(newMemberId, updatedMember.memberId());
+        assertEquals(Optional.of("group-instance-id"), updatedMember.groupInstanceId());
+        assertEquals(7000, updatedMember.rebalanceTimeoutMs());
+        assertEquals(4500, updatedMember.sessionTimeoutMs());
+        assertEquals(protocols, updatedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateWithUpdatedProtocolTriggersRebalance() throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. The selected protocol changes and triggers a rebalance.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array())
+        );
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertEquals(2, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateErrors() throws Exception {
+        // If the append future fails, we need to revert the soft state to the original member.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, false);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        GenericGroupMember oldMember = group.member(response.memberId());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID but the append fails. This reverts the soft state of the group.
+        protocols.add(new JoinGroupRequestProtocol()
+                .setName("roundrobin")
+                .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                    Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setSessionTimeoutMs(6000)
+                .setRebalanceTimeoutMs(7000)
+                .setProtocols(protocols),
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMember.memberId())
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code());
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        GenericGroupMember revertedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(oldMember.memberId(), revertedMember.memberId());
+        assertEquals(oldMember.groupInstanceId(), revertedMember.groupInstanceId());
+        assertEquals(oldMember.rebalanceTimeoutMs(), revertedMember.rebalanceTimeoutMs());
+        assertEquals(oldMember.sessionTimeoutMs(), revertedMember.sessionTimeoutMs());
+        assertEquals(oldMember.supportedProtocols(), revertedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInCompletingRebalanceStateTriggersRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID and triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);

Review Comment:
   I find the helpers a bit confusing. It is not clear what's the difference between `joinGenericGroup` and `sendGenericGroupJoin` for instance. Is it possible to simplify them?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2402,143 +2610,2090 @@ public void testOnNewMetadataImage() {
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = genericGroup(context, "group-id");
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = genericGroup(context, "group-id");
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
         }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+
+        // Send as static member join.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setGroupInstanceId("group-instance-id"), true, true, null);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(firstResponse.leader(), firstMemberId);
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, group.allMembers().iterator().next().memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+        assertNull(response.protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertFalse(responseFuture.isDone());
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+        context.timer.advanceClock(2);
+
+        // Advance clock past initial rebalance delay and verify futures are not completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+
+        // Advance clock beyond recomputed delay and make sure the futures have completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldDelayRebalanceUptoRebalanceTimeout() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs + 1);
+
+        CompletableFuture<JoinGroupResponseData> thirdMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        // Advance clock right before rebalance timeout.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+        assertFalse(thirdMemberResponseFuture.isDone());
+
+        // Advance clock beyond rebalance timeout.
+        context.timer.advanceClock(1);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertTrue(thirdMemberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), thirdMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupReplaceStaticMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .build();
+
+
+        // Send join group as static member.
+        CompletableFuture<JoinGroupResponseData> oldMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(oldMemberResponseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Replace static member with new member id. Old member id should be fenced.
+        CompletableFuture<JoinGroupResponseData> newMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        assertFalse(newMemberResponseFuture.isDone());
+        assertTrue(oldMemberResponseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Complete join for new member.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(newMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), newMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemovePendingMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(0, group.size());
+        assertEquals(1, group.numPendingJoinMembers());
+
+        // Advance clock by session timeout. Pending member should be removed from group as heartbeat expires.
+        context.timer.advanceClock(1000);
+        assertEquals(0, group.numPendingJoinMembers());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemoveMember() throws Exception {
+        // Set initial rebalance delay to simulate a long running rebalance.
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupInitialRebalanceDelayMs(10 * 60 * 1000)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        String memberId = group.leaderOrNull();
+        assertEquals(1, group.size());
+
+        // Advance clock by new member join timeout. Member should be removed from group as heartbeat expires.
+        // A group that transitions to Empty after completing join phase will generate records.
+        context.timer.expectResult(heartbeatKey("group-id", memberId), new ExpectedGenericGroupResult(
+            Collections.singletonList(newGroupMetadataRecord("group-id",
+                new GroupMetadataValue()
+                    .setMembers(Collections.emptyList())
+                    .setGeneration(1)
+                    .setLeader(null)
+                    .setProtocolType("consumer")
+                    .setProtocol(null)
+                    .setCurrentStateTimestamp(context.time.milliseconds()),
+                MetadataVersion.latest())),
+            Errors.NONE,
+            false
+        ));
+
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.size());
+    }
+
+    @Test
+    public void testExistingMemberJoinDeadGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertTrue(group.hasMemberId(memberId));
+
+        group.transitionTo(DEAD);
+
+        response = context.joinGenericGroupAsDynamicMember(request.setMemberId(memberId));
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupExistingPendingMemberWithGroupInstanceIdThrowsException() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertTrue(responseFuture.isDone());
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        assertThrows(IllegalStateException.class,
+            () -> context.sendGenericGroupJoin(request.setMemberId(memberId).setGroupInstanceId("group-instance-id"))
+        );
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        GenericGroupMember member = group.member(memberId);
+
+        assertEquals(protocols, member.supportedProtocols());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(1, group.generationId());
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        // Send updated member metadata. This should trigger a rebalance and complete the join phase.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(memberId)
+            .setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+        assertEquals(protocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingLeaderTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertTrue(group.isLeader(memberId));
+        assertEquals(1, group.generationId());
+
+        group.transitionTo(STABLE);
+        // Sending join group as leader should trigger a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with updated metadata. This should trigger a rebalance.
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        JoinGroupRequestData memberRequest = request.setMemberId(memberId).setProtocols(protocols);
+        memberResponseFuture = context.sendGenericGroupJoin(memberRequest);
+
+        assertFalse(memberResponseFuture.isDone());
+
+        // Leader rejoins. This completes the join group phase.
+        leaderResponseFuture = context.sendGenericGroupJoin(request.setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(3, group.generationId());
+        assertEquals(2, group.size());
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberDoesNotTriggerRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with no metadata changes. This does not trigger a rebalance.
+        memberResponseFuture = context.sendGenericGroupJoin(request.setMemberId(memberId));
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, memberResponseFuture.get(5, TimeUnit.SECONDS).generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberInEmptyState() throws Exception {
+        // Existing member joins a group that is in Empty/Dead state. Ask member to rejoin with generation id reset.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(-1, responseFuture.get(5, TimeUnit.SECONDS).generationId());
+    }
+
+    @Test
+    public void testCompleteJoinRemoveNotYetRejoinedDynamicMembers() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.generationId());
+
+        // Add new member. This triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout. This will expire the leader as it has not rejoined.
+        context.timer.advanceClock(1000);
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertTrue(group.hasMemberId(memberResponseFuture.get(5, TimeUnit.SECONDS).memberId()));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseInEmptyStateSkipsRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+        assertEquals(0, group.generationId());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        group.transitionTo(DEAD);
+
+        // Advance clock by initial rebalance delay to complete join phase.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertEquals(0, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseNoMembersRejoinedExtendsJoinPhase() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("first-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(30000)
+            .withRebalanceTimeoutMs(10000)
+            .build();
+
+        // First member joins group and completes join phase.
+        JoinGroupResponseData firstMemberResponse = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), firstMemberResponse.errorCode());
+        String firstMemberId = firstMemberResponse.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        // Second member joins and group goes into rebalancing state.
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setGroupInstanceId("second-instance-id"));
+
+        // First static member rejoins and completes join phase.
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        String secondMemberId = secondMemberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        // Trigger a rebalance. No members rejoined.
+        context.groupMetadataManager.prepareRebalance(group, "trigger rebalance");
+
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(0, group.numAwaitingJoinResponse());
+
+        // Advance clock by rebalance timeout to complete join phase. As long as both members have not
+        // rejoined, we extend the join phase.
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        // Let first and second member rejoin. This should complete the join phase.
+        firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(secondMemberId)
+            .setGroupInstanceId("second-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(3, group.generationId());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testReplaceStaticMemberInStableStateNoError(
+        boolean supportSkippingAssignment
+    ) throws Exception {
+

Review Comment:
   nit: empty line.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2402,143 +2610,2090 @@ public void testOnNewMetadataImage() {
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = genericGroup(context, "group-id");
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = genericGroup(context, "group-id");
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
         }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+
+        // Send as static member join.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setGroupInstanceId("group-instance-id"), true, true, null);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(firstResponse.leader(), firstMemberId);
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, group.allMembers().iterator().next().memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+        assertNull(response.protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertFalse(responseFuture.isDone());
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+        context.timer.advanceClock(2);
+
+        // Advance clock past initial rebalance delay and verify futures are not completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+
+        // Advance clock beyond recomputed delay and make sure the futures have completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldDelayRebalanceUptoRebalanceTimeout() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs + 1);
+
+        CompletableFuture<JoinGroupResponseData> thirdMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        // Advance clock right before rebalance timeout.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+        assertFalse(thirdMemberResponseFuture.isDone());
+
+        // Advance clock beyond rebalance timeout.
+        context.timer.advanceClock(1);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertTrue(thirdMemberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), thirdMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupReplaceStaticMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .build();
+
+
+        // Send join group as static member.
+        CompletableFuture<JoinGroupResponseData> oldMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(oldMemberResponseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Replace static member with new member id. Old member id should be fenced.
+        CompletableFuture<JoinGroupResponseData> newMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        assertFalse(newMemberResponseFuture.isDone());
+        assertTrue(oldMemberResponseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Complete join for new member.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(newMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), newMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemovePendingMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(0, group.size());
+        assertEquals(1, group.numPendingJoinMembers());
+
+        // Advance clock by session timeout. Pending member should be removed from group as heartbeat expires.
+        context.timer.advanceClock(1000);
+        assertEquals(0, group.numPendingJoinMembers());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemoveMember() throws Exception {
+        // Set initial rebalance delay to simulate a long running rebalance.
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupInitialRebalanceDelayMs(10 * 60 * 1000)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        String memberId = group.leaderOrNull();
+        assertEquals(1, group.size());
+
+        // Advance clock by new member join timeout. Member should be removed from group as heartbeat expires.
+        // A group that transitions to Empty after completing join phase will generate records.
+        context.timer.expectResult(heartbeatKey("group-id", memberId), new ExpectedGenericGroupResult(
+            Collections.singletonList(newGroupMetadataRecord("group-id",
+                new GroupMetadataValue()
+                    .setMembers(Collections.emptyList())
+                    .setGeneration(1)
+                    .setLeader(null)
+                    .setProtocolType("consumer")
+                    .setProtocol(null)
+                    .setCurrentStateTimestamp(context.time.milliseconds()),
+                MetadataVersion.latest())),
+            Errors.NONE,
+            false
+        ));
+
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.size());
+    }
+
+    @Test
+    public void testExistingMemberJoinDeadGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertTrue(group.hasMemberId(memberId));
+
+        group.transitionTo(DEAD);
+
+        response = context.joinGenericGroupAsDynamicMember(request.setMemberId(memberId));
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupExistingPendingMemberWithGroupInstanceIdThrowsException() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertTrue(responseFuture.isDone());
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        assertThrows(IllegalStateException.class,
+            () -> context.sendGenericGroupJoin(request.setMemberId(memberId).setGroupInstanceId("group-instance-id"))
+        );
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        GenericGroupMember member = group.member(memberId);
+
+        assertEquals(protocols, member.supportedProtocols());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(1, group.generationId());
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        // Send updated member metadata. This should trigger a rebalance and complete the join phase.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(memberId)
+            .setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+        assertEquals(protocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingLeaderTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertTrue(group.isLeader(memberId));
+        assertEquals(1, group.generationId());
+
+        group.transitionTo(STABLE);
+        // Sending join group as leader should trigger a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with updated metadata. This should trigger a rebalance.
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        JoinGroupRequestData memberRequest = request.setMemberId(memberId).setProtocols(protocols);
+        memberResponseFuture = context.sendGenericGroupJoin(memberRequest);
+
+        assertFalse(memberResponseFuture.isDone());
+
+        // Leader rejoins. This completes the join group phase.
+        leaderResponseFuture = context.sendGenericGroupJoin(request.setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(3, group.generationId());
+        assertEquals(2, group.size());
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberDoesNotTriggerRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with no metadata changes. This does not trigger a rebalance.
+        memberResponseFuture = context.sendGenericGroupJoin(request.setMemberId(memberId));
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, memberResponseFuture.get(5, TimeUnit.SECONDS).generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberInEmptyState() throws Exception {
+        // Existing member joins a group that is in Empty/Dead state. Ask member to rejoin with generation id reset.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(-1, responseFuture.get(5, TimeUnit.SECONDS).generationId());
+    }
+
+    @Test
+    public void testCompleteJoinRemoveNotYetRejoinedDynamicMembers() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.generationId());
+
+        // Add new member. This triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout. This will expire the leader as it has not rejoined.
+        context.timer.advanceClock(1000);
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertTrue(group.hasMemberId(memberResponseFuture.get(5, TimeUnit.SECONDS).memberId()));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseInEmptyStateSkipsRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+        assertEquals(0, group.generationId());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        group.transitionTo(DEAD);
+
+        // Advance clock by initial rebalance delay to complete join phase.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertEquals(0, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseNoMembersRejoinedExtendsJoinPhase() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("first-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(30000)
+            .withRebalanceTimeoutMs(10000)
+            .build();
+
+        // First member joins group and completes join phase.
+        JoinGroupResponseData firstMemberResponse = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), firstMemberResponse.errorCode());
+        String firstMemberId = firstMemberResponse.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        // Second member joins and group goes into rebalancing state.
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setGroupInstanceId("second-instance-id"));
+
+        // First static member rejoins and completes join phase.
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        String secondMemberId = secondMemberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        // Trigger a rebalance. No members rejoined.
+        context.groupMetadataManager.prepareRebalance(group, "trigger rebalance");
+
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(0, group.numAwaitingJoinResponse());
+
+        // Advance clock by rebalance timeout to complete join phase. As long as both members have not
+        // rejoined, we extend the join phase.
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        // Let first and second member rejoin. This should complete the join phase.
+        firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(secondMemberId)
+            .setGroupInstanceId("second-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(3, group.generationId());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testReplaceStaticMemberInStableStateNoError(
+        boolean supportSkippingAssignment
+    ) throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, supportSkippingAssignment);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String oldMemberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. This should update the log with the generated member id.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request
+                .setProtocols(protocols)
+                .setRebalanceTimeoutMs(7000)
+                .setSessionTimeoutMs(4500),
+            true,
+            supportSkippingAssignment,
+            new ExpectedGenericGroupResult(Errors.NONE, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        String newMemberId = group.staticMemberId("group-instance-id");
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMemberId)
+            .setMemberId(newMemberId)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(supportSkippingAssignment)
+            .setErrorCode(Errors.NONE.code());
+
+        if (supportSkippingAssignment) {
+            expectedResponse
+                .setMembers(Collections.singletonList(
+                    new JoinGroupResponseData.JoinGroupResponseMember()
+                        .setMemberId(newMemberId)
+                        .setGroupInstanceId("group-instance-id")
+                        .setMetadata(protocols.find("range").metadata())
+                    ))
+                .setLeader(newMemberId);
+        }
+
+        GenericGroupMember updatedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        assertEquals(newMemberId, updatedMember.memberId());
+        assertEquals(Optional.of("group-instance-id"), updatedMember.groupInstanceId());
+        assertEquals(7000, updatedMember.rebalanceTimeoutMs());
+        assertEquals(4500, updatedMember.sessionTimeoutMs());
+        assertEquals(protocols, updatedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateWithUpdatedProtocolTriggersRebalance() throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. The selected protocol changes and triggers a rebalance.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array())
+        );
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertEquals(2, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateErrors() throws Exception {
+        // If the append future fails, we need to revert the soft state to the original member.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, false);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        GenericGroupMember oldMember = group.member(response.memberId());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID but the append fails. This reverts the soft state of the group.
+        protocols.add(new JoinGroupRequestProtocol()
+                .setName("roundrobin")
+                .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                    Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setSessionTimeoutMs(6000)
+                .setRebalanceTimeoutMs(7000)
+                .setProtocols(protocols),
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMember.memberId())
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code());
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        GenericGroupMember revertedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(oldMember.memberId(), revertedMember.memberId());
+        assertEquals(oldMember.groupInstanceId(), revertedMember.groupInstanceId());
+        assertEquals(oldMember.rebalanceTimeoutMs(), revertedMember.rebalanceTimeoutMs());
+        assertEquals(oldMember.sessionTimeoutMs(), revertedMember.sessionTimeoutMs());
+        assertEquals(oldMember.supportedProtocols(), revertedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInCompletingRebalanceStateTriggersRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID and triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertEquals(2, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+    }
+
+    private <T> void assertUnorderedListEquals(
+        List<T> expected,
+        List<T> actual
+    ) {
+        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
+    }
+
+    private void assertResponseEquals(
+        ConsumerGroupHeartbeatResponseData expected,
+        ConsumerGroupHeartbeatResponseData actual
+    ) {
+        if (!responseEquals(expected, actual)) {
+            assertionFailure()
+                .expected(expected)
+                .actual(actual)
+                .buildAndThrow();
+        }
+    }
+
+    private boolean responseEquals(
+        ConsumerGroupHeartbeatResponseData expected,
+        ConsumerGroupHeartbeatResponseData actual
+    ) {
+        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
+        if (expected.errorCode() != actual.errorCode()) return false;
+        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
+        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
+        if (expected.memberEpoch() != actual.memberEpoch()) return false;
+        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
+        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
+        // Unordered comparison of the assignments.
+        return responseAssignmentEquals(expected.assignment(), actual.assignment());
+    }
+
+    private boolean responseAssignmentEquals(
+        ConsumerGroupHeartbeatResponseData.Assignment expected,
+        ConsumerGroupHeartbeatResponseData.Assignment actual
+    ) {
+        if (expected == actual) return true;
+        if (expected == null) return false;
+        if (actual == null) return false;
+
+        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
+            return false;
+
+        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+    }
+
+    private Map<Uuid, Set<Integer>> fromAssignment(
+        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
+    ) {
+        if (assignment == null) return null;
+
+        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
+        assignment.forEach(topicPartitions -> {
+            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+        });
+        return assignmentMap;
+    }
+
+    private void assertRecordsEquals(
+        List<Record> expectedRecords,
+        List<Record> actualRecords
+    ) {
+        try {
+            assertEquals(expectedRecords.size(), actualRecords.size());
+
+            for (int i = 0; i < expectedRecords.size(); i++) {
+                Record expectedRecord = expectedRecords.get(i);
+                Record actualRecord = actualRecords.get(i);
+                assertRecordEquals(expectedRecord, actualRecord);
+            }
+        } catch (AssertionFailedError e) {
+            assertionFailure()
+                .expected(expectedRecords)
+                .actual(actualRecords)
+                .buildAndThrow();
+        }
+    }
+
+    private void assertRecordEquals(
+        Record expected,
+        Record actual
+    ) {
+        try {
+            assertApiMessageAndVersionEquals(expected.key(), actual.key());
+            assertApiMessageAndVersionEquals(expected.value(), actual.value());
+        } catch (AssertionFailedError e) {
+            assertionFailure()
+                .expected(expected)
+                .actual(actual)
+                .buildAndThrow();
+        }
+    }
+
+    private void assertApiMessageAndVersionEquals(
+        ApiMessageAndVersion expected,
+        ApiMessageAndVersion actual
+    ) {
+        if (expected == actual) return;
+
+        assertEquals(expected.version(), actual.version());
+
+        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
+            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
+            // always guaranteed. Therefore, we need a special comparator.
+            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
+                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
+            ConsumerGroupCurrentMemberAssignmentValue actualValue =
+                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+
+            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
+            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
+            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
+            assertEquals(expectedValue.error(), actualValue.error());
+            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
+            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+
+            // We transform those to Maps before comparing them.
+            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
+                fromTopicPartitions(actualValue.assignedPartitions()));
+            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
+                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
+            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
+                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
+        } else {
+            assertEquals(expected.message(), actual.message());
+        }
+    }
+
+    private Map<Uuid, Set<Integer>> fromTopicPartitions(
+        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
+    ) {
+        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
+        assignment.forEach(topicPartitions -> {
+            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
         });
         return assignmentMap;
     }
+
+    private static GenericGroup genericGroup(GroupMetadataManagerTestContext context, String groupId) {
+        return context.groupMetadataManager.getOrMaybeCreateGenericGroup(groupId, false);
+    }
+
+    private List<String> verifyGenericGroupJoinResponses(
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures,
+        int expectedSuccessCount,
+        Errors expectedFailure
+    ) {
+        int successCount = 0;
+        List<String> memberIds = new ArrayList<>();
+        for (CompletableFuture<JoinGroupResponseData> responseFuture : responseFutures) {
+            if (!responseFuture.isDone()) {
+                fail("All responseFutures should be completed.");
+            }
+            try {
+                if (responseFuture.get(5, TimeUnit.SECONDS).errorCode() == Errors.NONE.code()) {
+                    successCount++;
+                } else {
+                    assertEquals(
+                        expectedFailure.code(),
+                        responseFuture.get(5, TimeUnit.SECONDS).errorCode()
+                    );
+                }
+                memberIds.add(responseFuture.get(5, TimeUnit.SECONDS).memberId());
+            } catch (Exception e) {
+                fail("Unexpected exception: " + e.getMessage());
+            }
+        }
+
+        assertEquals(expectedSuccessCount, successCount);
+        return memberIds;
+    }
+
+    /**
+     * Verify the records that should be appended and complete the append future based on a
+     * configured error. Run any assertions to verify the result of the future completion.
+     *
+     * @param expectedResult  The expected result to compare against.
+     * @param result          The result from expiring a join/heartbeat/sync operation.
+     */
+    private static void verifyCoordinatorResult(
+        ExpectedGenericGroupResult expectedResult,
+        CoordinatorResult<Void, Record> result
+    ) {
+        if (expectedResult == null) {
+            assertEquals(EMPTY_RESULT, result);
+        } else {
+            assertEquals(expectedResult.records, result.records());
+            if (expectedResult.mockError == Errors.NONE) {
+                result.appendFuture().complete(null);
+            } else {
+                result.appendFuture().completeExceptionally(expectedResult.mockError.exception());
+            }
+        }
+    }
+
+    private static class MockCoordinatorTimer

Review Comment:
   We need to align on this one as I also have an implementation [here](https://github.com/apache/kafka/pull/13963). They look pretty close but they are different.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));

Review Comment:
   Yes. I would actually create a special test to validate this case and simplify all the others.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/RecordHelpers.java:
##########
@@ -436,6 +437,36 @@ public static Record newGroupMetadataTombstoneRecord(
         );
     }
 
+    /**
+     * Creates an empty GroupMetadata record.
+     *
+     * @param group              The generic group.
+     * @param metadataVersion    The metadata version.
+     * @return The record.
+     */
+    public static Record newEmptyGroupMetadataRecord(

Review Comment:
   Do we need to add a test for this one?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2402,143 +2610,2090 @@ public void testOnNewMetadataImage() {
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = genericGroup(context, "group-id");
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = genericGroup(context, "group-id");
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
         }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+
+        // Send as static member join.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setGroupInstanceId("group-instance-id"), true, true, null);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(firstResponse.leader(), firstMemberId);
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, group.allMembers().iterator().next().memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+        assertNull(response.protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertFalse(responseFuture.isDone());
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+        context.timer.advanceClock(2);
+
+        // Advance clock past initial rebalance delay and verify futures are not completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+
+        // Advance clock beyond recomputed delay and make sure the futures have completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldDelayRebalanceUptoRebalanceTimeout() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs + 1);
+
+        CompletableFuture<JoinGroupResponseData> thirdMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        // Advance clock right before rebalance timeout.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+        assertFalse(thirdMemberResponseFuture.isDone());
+
+        // Advance clock beyond rebalance timeout.
+        context.timer.advanceClock(1);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertTrue(thirdMemberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), thirdMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupReplaceStaticMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .build();
+
+
+        // Send join group as static member.
+        CompletableFuture<JoinGroupResponseData> oldMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(oldMemberResponseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Replace static member with new member id. Old member id should be fenced.
+        CompletableFuture<JoinGroupResponseData> newMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        assertFalse(newMemberResponseFuture.isDone());
+        assertTrue(oldMemberResponseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Complete join for new member.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(newMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), newMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemovePendingMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(0, group.size());
+        assertEquals(1, group.numPendingJoinMembers());
+
+        // Advance clock by session timeout. Pending member should be removed from group as heartbeat expires.
+        context.timer.advanceClock(1000);
+        assertEquals(0, group.numPendingJoinMembers());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemoveMember() throws Exception {
+        // Set initial rebalance delay to simulate a long running rebalance.
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupInitialRebalanceDelayMs(10 * 60 * 1000)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        String memberId = group.leaderOrNull();
+        assertEquals(1, group.size());
+
+        // Advance clock by new member join timeout. Member should be removed from group as heartbeat expires.
+        // A group that transitions to Empty after completing join phase will generate records.
+        context.timer.expectResult(heartbeatKey("group-id", memberId), new ExpectedGenericGroupResult(
+            Collections.singletonList(newGroupMetadataRecord("group-id",
+                new GroupMetadataValue()
+                    .setMembers(Collections.emptyList())
+                    .setGeneration(1)
+                    .setLeader(null)
+                    .setProtocolType("consumer")
+                    .setProtocol(null)
+                    .setCurrentStateTimestamp(context.time.milliseconds()),
+                MetadataVersion.latest())),
+            Errors.NONE,
+            false
+        ));
+
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.size());
+    }
+
+    @Test
+    public void testExistingMemberJoinDeadGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertTrue(group.hasMemberId(memberId));
+
+        group.transitionTo(DEAD);
+
+        response = context.joinGenericGroupAsDynamicMember(request.setMemberId(memberId));
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupExistingPendingMemberWithGroupInstanceIdThrowsException() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertTrue(responseFuture.isDone());
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        assertThrows(IllegalStateException.class,
+            () -> context.sendGenericGroupJoin(request.setMemberId(memberId).setGroupInstanceId("group-instance-id"))
+        );
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        GenericGroupMember member = group.member(memberId);
+
+        assertEquals(protocols, member.supportedProtocols());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(1, group.generationId());
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        // Send updated member metadata. This should trigger a rebalance and complete the join phase.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(memberId)
+            .setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+        assertEquals(protocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingLeaderTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertTrue(group.isLeader(memberId));
+        assertEquals(1, group.generationId());
+
+        group.transitionTo(STABLE);
+        // Sending join group as leader should trigger a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with updated metadata. This should trigger a rebalance.
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        JoinGroupRequestData memberRequest = request.setMemberId(memberId).setProtocols(protocols);
+        memberResponseFuture = context.sendGenericGroupJoin(memberRequest);
+
+        assertFalse(memberResponseFuture.isDone());
+
+        // Leader rejoins. This completes the join group phase.
+        leaderResponseFuture = context.sendGenericGroupJoin(request.setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(3, group.generationId());
+        assertEquals(2, group.size());
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberDoesNotTriggerRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with no metadata changes. This does not trigger a rebalance.
+        memberResponseFuture = context.sendGenericGroupJoin(request.setMemberId(memberId));
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, memberResponseFuture.get(5, TimeUnit.SECONDS).generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberInEmptyState() throws Exception {
+        // Existing member joins a group that is in Empty/Dead state. Ask member to rejoin with generation id reset.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(-1, responseFuture.get(5, TimeUnit.SECONDS).generationId());
+    }
+
+    @Test
+    public void testCompleteJoinRemoveNotYetRejoinedDynamicMembers() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.generationId());
+
+        // Add new member. This triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout. This will expire the leader as it has not rejoined.
+        context.timer.advanceClock(1000);
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertTrue(group.hasMemberId(memberResponseFuture.get(5, TimeUnit.SECONDS).memberId()));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseInEmptyStateSkipsRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+        assertEquals(0, group.generationId());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        group.transitionTo(DEAD);
+
+        // Advance clock by initial rebalance delay to complete join phase.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertEquals(0, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseNoMembersRejoinedExtendsJoinPhase() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("first-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(30000)
+            .withRebalanceTimeoutMs(10000)
+            .build();
+
+        // First member joins group and completes join phase.
+        JoinGroupResponseData firstMemberResponse = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), firstMemberResponse.errorCode());
+        String firstMemberId = firstMemberResponse.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        // Second member joins and group goes into rebalancing state.
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setGroupInstanceId("second-instance-id"));
+
+        // First static member rejoins and completes join phase.
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        String secondMemberId = secondMemberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        // Trigger a rebalance. No members rejoined.
+        context.groupMetadataManager.prepareRebalance(group, "trigger rebalance");
+
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(0, group.numAwaitingJoinResponse());
+
+        // Advance clock by rebalance timeout to complete join phase. As long as both members have not
+        // rejoined, we extend the join phase.
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        // Let first and second member rejoin. This should complete the join phase.
+        firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(secondMemberId)
+            .setGroupInstanceId("second-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(3, group.generationId());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testReplaceStaticMemberInStableStateNoError(
+        boolean supportSkippingAssignment
+    ) throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, supportSkippingAssignment);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String oldMemberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. This should update the log with the generated member id.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request
+                .setProtocols(protocols)
+                .setRebalanceTimeoutMs(7000)
+                .setSessionTimeoutMs(4500),
+            true,
+            supportSkippingAssignment,
+            new ExpectedGenericGroupResult(Errors.NONE, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        String newMemberId = group.staticMemberId("group-instance-id");
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMemberId)
+            .setMemberId(newMemberId)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(supportSkippingAssignment)
+            .setErrorCode(Errors.NONE.code());
+
+        if (supportSkippingAssignment) {
+            expectedResponse
+                .setMembers(Collections.singletonList(
+                    new JoinGroupResponseData.JoinGroupResponseMember()
+                        .setMemberId(newMemberId)
+                        .setGroupInstanceId("group-instance-id")
+                        .setMetadata(protocols.find("range").metadata())
+                    ))
+                .setLeader(newMemberId);
+        }
+
+        GenericGroupMember updatedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        assertEquals(newMemberId, updatedMember.memberId());
+        assertEquals(Optional.of("group-instance-id"), updatedMember.groupInstanceId());
+        assertEquals(7000, updatedMember.rebalanceTimeoutMs());
+        assertEquals(4500, updatedMember.sessionTimeoutMs());
+        assertEquals(protocols, updatedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateWithUpdatedProtocolTriggersRebalance() throws Exception {
+

Review Comment:
   nit: empty line.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2402,143 +2610,2090 @@ public void testOnNewMetadataImage() {
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = genericGroup(context, "group-id");
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = genericGroup(context, "group-id");
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
         }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+
+        // Send as static member join.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setGroupInstanceId("group-instance-id"), true, true, null);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(firstResponse.leader(), firstMemberId);
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, group.allMembers().iterator().next().memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+        assertNull(response.protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertFalse(responseFuture.isDone());
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+        context.timer.advanceClock(2);
+
+        // Advance clock past initial rebalance delay and verify futures are not completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+
+        // Advance clock beyond recomputed delay and make sure the futures have completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldDelayRebalanceUptoRebalanceTimeout() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs + 1);
+
+        CompletableFuture<JoinGroupResponseData> thirdMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        // Advance clock right before rebalance timeout.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+        assertFalse(thirdMemberResponseFuture.isDone());
+
+        // Advance clock beyond rebalance timeout.
+        context.timer.advanceClock(1);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertTrue(thirdMemberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), thirdMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupReplaceStaticMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .build();
+
+
+        // Send join group as static member.
+        CompletableFuture<JoinGroupResponseData> oldMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(oldMemberResponseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Replace static member with new member id. Old member id should be fenced.
+        CompletableFuture<JoinGroupResponseData> newMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        assertFalse(newMemberResponseFuture.isDone());
+        assertTrue(oldMemberResponseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Complete join for new member.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(newMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), newMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemovePendingMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(0, group.size());
+        assertEquals(1, group.numPendingJoinMembers());
+
+        // Advance clock by session timeout. Pending member should be removed from group as heartbeat expires.
+        context.timer.advanceClock(1000);
+        assertEquals(0, group.numPendingJoinMembers());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemoveMember() throws Exception {
+        // Set initial rebalance delay to simulate a long running rebalance.
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupInitialRebalanceDelayMs(10 * 60 * 1000)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        String memberId = group.leaderOrNull();
+        assertEquals(1, group.size());
+
+        // Advance clock by new member join timeout. Member should be removed from group as heartbeat expires.
+        // A group that transitions to Empty after completing join phase will generate records.
+        context.timer.expectResult(heartbeatKey("group-id", memberId), new ExpectedGenericGroupResult(
+            Collections.singletonList(newGroupMetadataRecord("group-id",
+                new GroupMetadataValue()
+                    .setMembers(Collections.emptyList())
+                    .setGeneration(1)
+                    .setLeader(null)
+                    .setProtocolType("consumer")
+                    .setProtocol(null)
+                    .setCurrentStateTimestamp(context.time.milliseconds()),
+                MetadataVersion.latest())),
+            Errors.NONE,
+            false
+        ));
+
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.size());
+    }
+
+    @Test
+    public void testExistingMemberJoinDeadGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertTrue(group.hasMemberId(memberId));
+
+        group.transitionTo(DEAD);
+
+        response = context.joinGenericGroupAsDynamicMember(request.setMemberId(memberId));
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupExistingPendingMemberWithGroupInstanceIdThrowsException() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertTrue(responseFuture.isDone());
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        assertThrows(IllegalStateException.class,
+            () -> context.sendGenericGroupJoin(request.setMemberId(memberId).setGroupInstanceId("group-instance-id"))
+        );
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        GenericGroupMember member = group.member(memberId);
+
+        assertEquals(protocols, member.supportedProtocols());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(1, group.generationId());
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        // Send updated member metadata. This should trigger a rebalance and complete the join phase.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(memberId)
+            .setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+        assertEquals(protocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingLeaderTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertTrue(group.isLeader(memberId));
+        assertEquals(1, group.generationId());
+
+        group.transitionTo(STABLE);
+        // Sending join group as leader should trigger a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with updated metadata. This should trigger a rebalance.
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        JoinGroupRequestData memberRequest = request.setMemberId(memberId).setProtocols(protocols);
+        memberResponseFuture = context.sendGenericGroupJoin(memberRequest);
+
+        assertFalse(memberResponseFuture.isDone());
+
+        // Leader rejoins. This completes the join group phase.
+        leaderResponseFuture = context.sendGenericGroupJoin(request.setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(3, group.generationId());
+        assertEquals(2, group.size());
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberDoesNotTriggerRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with no metadata changes. This does not trigger a rebalance.
+        memberResponseFuture = context.sendGenericGroupJoin(request.setMemberId(memberId));
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, memberResponseFuture.get(5, TimeUnit.SECONDS).generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberInEmptyState() throws Exception {
+        // Existing member joins a group that is in Empty/Dead state. Ask member to rejoin with generation id reset.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(-1, responseFuture.get(5, TimeUnit.SECONDS).generationId());
+    }
+
+    @Test
+    public void testCompleteJoinRemoveNotYetRejoinedDynamicMembers() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.generationId());
+
+        // Add new member. This triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout. This will expire the leader as it has not rejoined.
+        context.timer.advanceClock(1000);
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertTrue(group.hasMemberId(memberResponseFuture.get(5, TimeUnit.SECONDS).memberId()));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseInEmptyStateSkipsRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+        assertEquals(0, group.generationId());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        group.transitionTo(DEAD);
+
+        // Advance clock by initial rebalance delay to complete join phase.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertEquals(0, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseNoMembersRejoinedExtendsJoinPhase() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("first-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(30000)
+            .withRebalanceTimeoutMs(10000)
+            .build();
+
+        // First member joins group and completes join phase.
+        JoinGroupResponseData firstMemberResponse = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), firstMemberResponse.errorCode());
+        String firstMemberId = firstMemberResponse.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        // Second member joins and group goes into rebalancing state.
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setGroupInstanceId("second-instance-id"));
+
+        // First static member rejoins and completes join phase.
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        String secondMemberId = secondMemberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        // Trigger a rebalance. No members rejoined.
+        context.groupMetadataManager.prepareRebalance(group, "trigger rebalance");
+
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(0, group.numAwaitingJoinResponse());
+
+        // Advance clock by rebalance timeout to complete join phase. As long as both members have not
+        // rejoined, we extend the join phase.
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        // Let first and second member rejoin. This should complete the join phase.
+        firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(secondMemberId)
+            .setGroupInstanceId("second-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(3, group.generationId());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testReplaceStaticMemberInStableStateNoError(
+        boolean supportSkippingAssignment
+    ) throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, supportSkippingAssignment);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String oldMemberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. This should update the log with the generated member id.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request
+                .setProtocols(protocols)
+                .setRebalanceTimeoutMs(7000)
+                .setSessionTimeoutMs(4500),
+            true,
+            supportSkippingAssignment,
+            new ExpectedGenericGroupResult(Errors.NONE, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        String newMemberId = group.staticMemberId("group-instance-id");
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMemberId)
+            .setMemberId(newMemberId)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(supportSkippingAssignment)
+            .setErrorCode(Errors.NONE.code());
+
+        if (supportSkippingAssignment) {
+            expectedResponse
+                .setMembers(Collections.singletonList(
+                    new JoinGroupResponseData.JoinGroupResponseMember()
+                        .setMemberId(newMemberId)
+                        .setGroupInstanceId("group-instance-id")
+                        .setMetadata(protocols.find("range").metadata())
+                    ))
+                .setLeader(newMemberId);
+        }
+
+        GenericGroupMember updatedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        assertEquals(newMemberId, updatedMember.memberId());
+        assertEquals(Optional.of("group-instance-id"), updatedMember.groupInstanceId());
+        assertEquals(7000, updatedMember.rebalanceTimeoutMs());
+        assertEquals(4500, updatedMember.sessionTimeoutMs());
+        assertEquals(protocols, updatedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateWithUpdatedProtocolTriggersRebalance() throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));

Review Comment:
   How do end up in this state here? Is there some code to advance the timer to complete the prepare phase?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2402,143 +2610,2090 @@ public void testOnNewMetadataImage() {
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = genericGroup(context, "group-id");
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = genericGroup(context, "group-id");
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
         }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+
+        // Static member
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withGroupInstanceId("group-instance-id")
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testGenericGroupJoinInconsistentProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("connect")
+            .withProtocols(protocols)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol().setName("range"));
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+
+        // Send as static member join.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setGroupInstanceId("group-instance-id"), true, true, null);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get().errorCode());
+    }
+
+    @Test
+    public void testJoinGroupWithEmptyGroupProtocol() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), response.errorCode());
+    }
+
+    @Test
+    public void testNewMemberJoinExpiration() throws Exception {
+        // This tests new member expiration during a protracted rebalance. We first create a
+        // group with one member which uses a large value for session timeout and rebalance timeout.
+        // We then join with one new member and let the rebalance hang while we await the first member.
+        // The new member join timeout expires and its JoinGroup request is failed.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000 + context.genericGroupNewMemberJoinTimeoutMs)
+            .withRebalanceTimeoutMs(2 * context.genericGroupNewMemberJoinTimeoutMs)
+            .build();
+
+        JoinGroupResponseData firstResponse = context.joinGenericGroupAsDynamicMember(request);
+        String firstMemberId = firstResponse.memberId();
+        assertEquals(firstResponse.leader(), firstMemberId);
+        assertEquals(Errors.NONE.code(), firstResponse.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertNotNull(group);
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        // Send second join group request for a new dynamic member.
+        CompletableFuture<JoinGroupResponseData> secondResponseFuture = context.sendGenericGroupJoin(request
+            .setSessionTimeoutMs(5000)
+            .setRebalanceTimeoutMs(5000));
+
+        assertFalse(secondResponseFuture.isDone());
+
+        assertEquals(2, group.allMembers().size());
+        assertEquals(1, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+
+        GenericGroupMember newMember = group.allMembers().stream().filter(GenericGroupMember::isNew).findFirst().get();
+        assertNotEquals(firstMemberId, newMember.memberId());
+
+        // Advance clock by new member join timeout to expire the second member.
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+
+        assertTrue(secondResponseFuture.isDone());
+
+        JoinGroupResponseData secondResponse = secondResponseFuture.get(5, TimeUnit.SECONDS);
+
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), secondResponse.errorCode());
+        assertEquals(1, group.allMembers().size());
+        assertEquals(0, group.allMembers().stream().filter(GenericGroupMember::isNew).count());
+        assertEquals(firstMemberId, group.allMembers().iterator().next().memberId());
+    }
+
+    @Test
+    public void testJoinGroupInconsistentGroupProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        JoinGroupRequestProtocolCollection otherProtocols = new JoinGroupRequestProtocolCollection(0);
+        otherProtocols.add(new JoinGroupRequestProtocol().setName("roundrobin"));
+        CompletableFuture<JoinGroupResponseData> otherResponseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(otherProtocols)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), otherResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSecondJoinInconsistentProtocol() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending an inconsistent protocol should be refused
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+        JoinGroupRequestProtocolCollection emptyProtocols = new JoinGroupRequestProtocolCollection(0);
+        request = request.setMemberId(memberId)
+            .setProtocols(emptyProtocols);
+
+        responseFuture = context.sendGenericGroupJoin(request, true);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INCONSISTENT_GROUP_PROTOCOL.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+
+        // Sending consistent protocol should be accepted
+        responseFuture = context.sendGenericGroupJoin(request.setProtocols(protocols), true);
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testStaticMemberJoinAsFirstMember() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testStaticMemberRejoinWithExplicitUnknownMemberId() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request.setMemberId("unknown-member-id"));
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerExistingGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .withRebalanceTimeoutMs(5000)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        JoinGroupResponseData otherResponse = context.joinGenericGroupAsDynamicMember(request
+            .setMemberId("other-member-id"));
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), otherResponse.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewDeadGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+        group.transitionTo(DEAD);
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupProtocolTypeIsNotProvidedWhenAnErrorOccurs() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), response.errorCode());
+        assertNull(response.protocolType());
+    }
+
+    @Test
+    public void testJoinGroupReturnsTheProtocolType() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        // Leader joins
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(leaderResponseFuture.isDone());
+
+        // Member joins
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+
+        // Complete join group phase
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", leaderResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals("consumer", memberResponseFuture.get(5, TimeUnit.SECONDS).protocolType());
+    }
+
+    @Test
+    public void shouldDelayInitialRebalanceByGroupInitialRebalanceDelayOnEmptyGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertFalse(responseFuture.isDone());
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldResetRebalanceDelayWhenNewMemberJoinsGroupDuringInitialRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 3)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+        context.timer.advanceClock(2);
+
+        // Advance clock past initial rebalance delay and verify futures are not completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2 + 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+
+        // Advance clock beyond recomputed delay and make sure the futures have completed.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs / 2);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void shouldDelayRebalanceUptoRebalanceTimeout() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withRebalanceTimeoutMs(context.genericGroupInitialRebalanceDelayMs * 2)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs + 1);
+
+        CompletableFuture<JoinGroupResponseData> thirdMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        // Advance clock right before rebalance timeout.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs - 1);
+        assertFalse(firstMemberResponseFuture.isDone());
+        assertFalse(secondMemberResponseFuture.isDone());
+        assertFalse(thirdMemberResponseFuture.isDone());
+
+        // Advance clock beyond rebalance timeout.
+        context.timer.advanceClock(1);
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertTrue(thirdMemberResponseFuture.isDone());
+
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), thirdMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupReplaceStaticMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withGroupInstanceId("group-instance-id")
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(5000)
+            .build();
+
+
+        // Send join group as static member.
+        CompletableFuture<JoinGroupResponseData> oldMemberResponseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(oldMemberResponseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Replace static member with new member id. Old member id should be fenced.
+        CompletableFuture<JoinGroupResponseData> newMemberResponseFuture = context.sendGenericGroupJoin(request);
+
+        assertFalse(newMemberResponseFuture.isDone());
+        assertTrue(oldMemberResponseFuture.isDone());
+        assertEquals(Errors.FENCED_INSTANCE_ID.code(), oldMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+
+        // Complete join for new member.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertTrue(newMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), newMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertEquals(1, group.size());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemovePendingMember() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.MEMBER_ID_REQUIRED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(0, group.size());
+        assertEquals(1, group.numPendingJoinMembers());
+
+        // Advance clock by session timeout. Pending member should be removed from group as heartbeat expires.
+        context.timer.advanceClock(1000);
+        assertEquals(0, group.numPendingJoinMembers());
+    }
+
+    @Test
+    public void testHeartbeatExpirationShouldRemoveMember() throws Exception {
+        // Set initial rebalance delay to simulate a long running rebalance.
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupInitialRebalanceDelayMs(10 * 60 * 1000)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        String memberId = group.leaderOrNull();
+        assertEquals(1, group.size());
+
+        // Advance clock by new member join timeout. Member should be removed from group as heartbeat expires.
+        // A group that transitions to Empty after completing join phase will generate records.
+        context.timer.expectResult(heartbeatKey("group-id", memberId), new ExpectedGenericGroupResult(
+            Collections.singletonList(newGroupMetadataRecord("group-id",
+                new GroupMetadataValue()
+                    .setMembers(Collections.emptyList())
+                    .setGeneration(1)
+                    .setLeader(null)
+                    .setProtocolType("consumer")
+                    .setProtocol(null)
+                    .setCurrentStateTimestamp(context.time.milliseconds()),
+                MetadataVersion.latest())),
+            Errors.NONE,
+            false
+        ));
+
+        context.timer.advanceClock(context.genericGroupNewMemberJoinTimeoutMs);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(0, group.size());
+    }
+
+    @Test
+    public void testExistingMemberJoinDeadGroup() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertTrue(group.hasMemberId(memberId));
+
+        group.transitionTo(DEAD);
+
+        response = context.joinGenericGroupAsDynamicMember(request.setMemberId(memberId));
+        assertEquals(Errors.COORDINATOR_NOT_AVAILABLE.code(), response.errorCode());
+    }
+
+    @Test
+    public void testJoinGroupExistingPendingMemberWithGroupInstanceIdThrowsException() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            true,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertTrue(responseFuture.isDone());
+        String memberId = responseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        assertThrows(IllegalStateException.class,
+            () -> context.sendGenericGroupJoin(request.setMemberId(memberId).setGroupInstanceId("group-instance-id"))
+        );
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberUpdatedMetadataTriggersRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        GenericGroupMember member = group.member(memberId);
+
+        assertEquals(protocols, member.supportedProtocols());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(1, group.generationId());
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        // Send updated member metadata. This should trigger a rebalance and complete the join phase.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(memberId)
+            .setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+        assertEquals(protocols, member.supportedProtocols());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingLeaderTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertTrue(group.isLeader(memberId));
+        assertEquals(1, group.generationId());
+
+        group.transitionTo(STABLE);
+        // Sending join group as leader should trigger a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testJoinGroupAsExistingMemberWithUpdatedMetadataTriggersRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), leaderResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with updated metadata. This should trigger a rebalance.
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array()));
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        JoinGroupRequestData memberRequest = request.setMemberId(memberId).setProtocols(protocols);
+        memberResponseFuture = context.sendGenericGroupJoin(memberRequest);
+
+        assertFalse(memberResponseFuture.isDone());
+
+        // Leader rejoins. This completes the join group phase.
+        leaderResponseFuture = context.sendGenericGroupJoin(request.setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(3, group.generationId());
+        assertEquals(2, group.size());
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberDoesNotTriggerRebalanceInStableState() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        String leaderId = leaderResponse.leader();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup("group-id", false);
+        assertEquals(1, group.generationId());
+
+        // Member joins.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        // Leader also rejoins. Completes join group phase.
+        CompletableFuture<JoinGroupResponseData> leaderResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(leaderId));
+
+        assertTrue(leaderResponseFuture.isDone());
+        assertTrue(memberResponseFuture.isDone());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        String memberId = memberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        group.transitionTo(STABLE);
+
+        // Member rejoins with no metadata changes. This does not trigger a rebalance.
+        memberResponseFuture = context.sendGenericGroupJoin(request.setMemberId(memberId));
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, memberResponseFuture.get(5, TimeUnit.SECONDS).generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testJoinGroupExistingMemberInEmptyState() throws Exception {
+        // Existing member joins a group that is in Empty/Dead state. Ask member to rejoin with generation id reset.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String memberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        group.transitionTo(PREPARING_REBALANCE);
+        group.transitionTo(EMPTY);
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setMemberId(memberId));
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.UNKNOWN_MEMBER_ID.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(-1, responseFuture.get(5, TimeUnit.SECONDS).generationId());
+    }
+
+    @Test
+    public void testCompleteJoinRemoveNotYetRejoinedDynamicMembers() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        JoinGroupResponseData leaderResponse = context.joinGenericGroupAsDynamicMember(request);
+        assertEquals(Errors.NONE.code(), leaderResponse.errorCode());
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.generationId());
+
+        // Add new member. This triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> memberResponseFuture = context.sendGenericGroupJoin(request);
+        assertFalse(memberResponseFuture.isDone());
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout. This will expire the leader as it has not rejoined.
+        context.timer.advanceClock(1000);
+
+        assertTrue(memberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), memberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertTrue(group.hasMemberId(memberResponseFuture.get(5, TimeUnit.SECONDS).memberId()));
+        assertEquals(2, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseInEmptyStateSkipsRebalance() {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(1000)
+            .withRebalanceTimeoutMs(1000)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request,
+            false,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        );
+        assertFalse(responseFuture.isDone());
+
+        GenericGroup group = genericGroup(context, "group-id");
+        assertEquals(0, group.generationId());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        group.transitionTo(DEAD);
+
+        // Advance clock by initial rebalance delay to complete join phase.
+        context.timer.advanceClock(context.genericGroupInitialRebalanceDelayMs);
+        assertEquals(0, group.generationId());
+    }
+
+    @Test
+    public void testCompleteJoinPhaseNoMembersRejoinedExtendsJoinPhase() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("first-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withSessionTimeoutMs(30000)
+            .withRebalanceTimeoutMs(10000)
+            .build();
+
+        // First member joins group and completes join phase.
+        JoinGroupResponseData firstMemberResponse = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), firstMemberResponse.errorCode());
+        String firstMemberId = firstMemberResponse.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        // Second member joins and group goes into rebalancing state.
+        CompletableFuture<JoinGroupResponseData> secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setGroupInstanceId("second-instance-id"));
+
+        // First static member rejoins and completes join phase.
+        CompletableFuture<JoinGroupResponseData> firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        String secondMemberId = secondMemberResponseFuture.get(5, TimeUnit.SECONDS).memberId();
+
+        // Trigger a rebalance. No members rejoined.
+        context.groupMetadataManager.prepareRebalance(group, "trigger rebalance");
+
+        assertEquals(2, group.size());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(0, group.numAwaitingJoinResponse());
+
+        // Advance clock by rebalance timeout to complete join phase. As long as both members have not
+        // rejoined, we extend the join phase.
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+        context.timer.advanceClock(10000);
+        assertEquals(10000, context.timer.operationsByKey.get("join-group-id").remainingMs);
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+        
+        // Let first and second member rejoin. This should complete the join phase.
+        firstMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(firstMemberId)
+            .setGroupInstanceId("first-instance-id"));
+
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(2, group.generationId());
+
+        secondMemberResponseFuture = context.sendGenericGroupJoin(request
+            .setMemberId(secondMemberId)
+            .setGroupInstanceId("second-instance-id"));
+
+        assertTrue(firstMemberResponseFuture.isDone());
+        assertTrue(secondMemberResponseFuture.isDone());
+        assertEquals(Errors.NONE.code(), firstMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(Errors.NONE.code(), secondMemberResponseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+        assertEquals(2, group.size());
+        assertEquals(3, group.generationId());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    public void testReplaceStaticMemberInStableStateNoError(
+        boolean supportSkippingAssignment
+    ) throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, supportSkippingAssignment);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+        String oldMemberId = response.memberId();
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. This should update the log with the generated member id.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request
+                .setProtocols(protocols)
+                .setRebalanceTimeoutMs(7000)
+                .setSessionTimeoutMs(4500),
+            true,
+            supportSkippingAssignment,
+            new ExpectedGenericGroupResult(Errors.NONE, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        String newMemberId = group.staticMemberId("group-instance-id");
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMemberId)
+            .setMemberId(newMemberId)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(supportSkippingAssignment)
+            .setErrorCode(Errors.NONE.code());
+
+        if (supportSkippingAssignment) {
+            expectedResponse
+                .setMembers(Collections.singletonList(
+                    new JoinGroupResponseData.JoinGroupResponseMember()
+                        .setMemberId(newMemberId)
+                        .setGroupInstanceId("group-instance-id")
+                        .setMetadata(protocols.find("range").metadata())
+                    ))
+                .setLeader(newMemberId);
+        }
+
+        GenericGroupMember updatedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        assertEquals(newMemberId, updatedMember.memberId());
+        assertEquals(Optional.of("group-instance-id"), updatedMember.groupInstanceId());
+        assertEquals(7000, updatedMember.rebalanceTimeoutMs());
+        assertEquals(4500, updatedMember.sessionTimeoutMs());
+        assertEquals(protocols, updatedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateWithUpdatedProtocolTriggersRebalance() throws Exception {
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID. The selected protocol changes and triggers a rebalance.
+        protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("roundrobin")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("bar"))).array())
+        );
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setProtocols(protocols)
+        );
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertEquals(2, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInStableStateErrors() throws Exception {
+        // If the append future fails, we need to revert the soft state to the original member.
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection(0);
+
+        protocols.add(new JoinGroupRequestProtocol()
+            .setName("range")
+            .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                Collections.singletonList("foo"))).array())
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withProtocolType("consumer")
+            .withProtocols(protocols)
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, false, false);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        GenericGroupMember oldMember = group.member(response.memberId());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Simulate successful sync group phase
+        group.transitionTo(STABLE);
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID but the append fails. This reverts the soft state of the group.
+        protocols.add(new JoinGroupRequestProtocol()
+                .setName("roundrobin")
+                .setMetadata(ConsumerProtocol.serializeSubscription(new ConsumerPartitionAssignor.Subscription(
+                    Collections.singletonList("bar"))).array()));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(
+            request.setSessionTimeoutMs(6000)
+                .setRebalanceTimeoutMs(7000)
+                .setProtocols(protocols),
+            false,
+            false,
+            new ExpectedGenericGroupResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, false)
+        );
+        assertTrue(responseFuture.isDone());
+
+        JoinGroupResponseData expectedResponse = new JoinGroupResponseData()
+            .setMembers(Collections.emptyList())
+            .setLeader(oldMember.memberId())
+            .setMemberId(UNKNOWN_MEMBER_ID)
+            .setGenerationId(1)
+            .setProtocolType("consumer")
+            .setProtocolName("range")
+            .setSkipAssignment(false)
+            .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code());
+
+        assertEquals(expectedResponse, responseFuture.get(5, TimeUnit.SECONDS));
+
+        GenericGroupMember revertedMember = group.member(group.staticMemberId("group-instance-id"));
+
+        assertEquals(oldMember.memberId(), revertedMember.memberId());
+        assertEquals(oldMember.groupInstanceId(), revertedMember.groupInstanceId());
+        assertEquals(oldMember.rebalanceTimeoutMs(), revertedMember.rebalanceTimeoutMs());
+        assertEquals(oldMember.sessionTimeoutMs(), revertedMember.sessionTimeoutMs());
+        assertEquals(oldMember.supportedProtocols(), revertedMember.supportedProtocols());
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(STABLE));
+    }
+
+    @Test
+    public void testReplaceStaticMemberInCompletingRebalanceStateTriggersRebalance() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withGroupInstanceId("group-instance-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroup(request, true, true);
+        assertEquals(Errors.NONE.code(), response.errorCode());
+
+        GenericGroup group = genericGroup(context, "group-id");
+
+        assertEquals(1, group.size());
+        assertEquals(1, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        // Static member rejoins with UNKNOWN_MEMBER_ID and triggers a rebalance.
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.NONE.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+        assertEquals(1, group.size());
+        assertEquals(2, group.generationId());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+    }
+
+    private <T> void assertUnorderedListEquals(
+        List<T> expected,
+        List<T> actual
+    ) {
+        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
+    }
+
+    private void assertResponseEquals(
+        ConsumerGroupHeartbeatResponseData expected,
+        ConsumerGroupHeartbeatResponseData actual
+    ) {
+        if (!responseEquals(expected, actual)) {
+            assertionFailure()
+                .expected(expected)
+                .actual(actual)
+                .buildAndThrow();
+        }
+    }
+
+    private boolean responseEquals(
+        ConsumerGroupHeartbeatResponseData expected,
+        ConsumerGroupHeartbeatResponseData actual
+    ) {
+        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
+        if (expected.errorCode() != actual.errorCode()) return false;
+        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
+        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
+        if (expected.memberEpoch() != actual.memberEpoch()) return false;
+        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
+        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
+        // Unordered comparison of the assignments.
+        return responseAssignmentEquals(expected.assignment(), actual.assignment());
+    }
+
+    private boolean responseAssignmentEquals(
+        ConsumerGroupHeartbeatResponseData.Assignment expected,
+        ConsumerGroupHeartbeatResponseData.Assignment actual
+    ) {
+        if (expected == actual) return true;
+        if (expected == null) return false;
+        if (actual == null) return false;
+
+        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
+            return false;
+
+        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+    }
+
+    private Map<Uuid, Set<Integer>> fromAssignment(
+        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
+    ) {
+        if (assignment == null) return null;
+
+        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
+        assignment.forEach(topicPartitions -> {
+            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
+        });
+        return assignmentMap;
+    }
+
+    private void assertRecordsEquals(
+        List<Record> expectedRecords,
+        List<Record> actualRecords
+    ) {
+        try {
+            assertEquals(expectedRecords.size(), actualRecords.size());
+
+            for (int i = 0; i < expectedRecords.size(); i++) {
+                Record expectedRecord = expectedRecords.get(i);
+                Record actualRecord = actualRecords.get(i);
+                assertRecordEquals(expectedRecord, actualRecord);
+            }
+        } catch (AssertionFailedError e) {
+            assertionFailure()
+                .expected(expectedRecords)
+                .actual(actualRecords)
+                .buildAndThrow();
+        }
+    }
+
+    private void assertRecordEquals(
+        Record expected,
+        Record actual
+    ) {
+        try {
+            assertApiMessageAndVersionEquals(expected.key(), actual.key());
+            assertApiMessageAndVersionEquals(expected.value(), actual.value());
+        } catch (AssertionFailedError e) {
+            assertionFailure()
+                .expected(expected)
+                .actual(actual)
+                .buildAndThrow();
+        }
+    }
+
+    private void assertApiMessageAndVersionEquals(
+        ApiMessageAndVersion expected,
+        ApiMessageAndVersion actual
+    ) {
+        if (expected == actual) return;
+
+        assertEquals(expected.version(), actual.version());
+
+        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
+            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
+            // always guaranteed. Therefore, we need a special comparator.
+            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
+                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
+            ConsumerGroupCurrentMemberAssignmentValue actualValue =
+                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+
+            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
+            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
+            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
+            assertEquals(expectedValue.error(), actualValue.error());
+            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
+            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+
+            // We transform those to Maps before comparing them.
+            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
+                fromTopicPartitions(actualValue.assignedPartitions()));
+            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
+                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
+            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
+                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
+        } else {
+            assertEquals(expected.message(), actual.message());
+        }
+    }
+
+    private Map<Uuid, Set<Integer>> fromTopicPartitions(
+        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
+    ) {
+        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
+        assignment.forEach(topicPartitions -> {
+            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
         });
         return assignmentMap;
     }
+
+    private static GenericGroup genericGroup(GroupMetadataManagerTestContext context, String groupId) {
+        return context.groupMetadataManager.getOrMaybeCreateGenericGroup(groupId, false);
+    }
+
+    private List<String> verifyGenericGroupJoinResponses(
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures,
+        int expectedSuccessCount,
+        Errors expectedFailure
+    ) {
+        int successCount = 0;
+        List<String> memberIds = new ArrayList<>();
+        for (CompletableFuture<JoinGroupResponseData> responseFuture : responseFutures) {
+            if (!responseFuture.isDone()) {
+                fail("All responseFutures should be completed.");
+            }
+            try {
+                if (responseFuture.get(5, TimeUnit.SECONDS).errorCode() == Errors.NONE.code()) {
+                    successCount++;
+                } else {
+                    assertEquals(
+                        expectedFailure.code(),
+                        responseFuture.get(5, TimeUnit.SECONDS).errorCode()
+                    );
+                }
+                memberIds.add(responseFuture.get(5, TimeUnit.SECONDS).memberId());
+            } catch (Exception e) {
+                fail("Unexpected exception: " + e.getMessage());
+            }
+        }
+
+        assertEquals(expectedSuccessCount, successCount);
+        return memberIds;
+    }
+
+    /**
+     * Verify the records that should be appended and complete the append future based on a
+     * configured error. Run any assertions to verify the result of the future completion.
+     *
+     * @param expectedResult  The expected result to compare against.
+     * @param result          The result from expiring a join/heartbeat/sync operation.
+     */
+    private static void verifyCoordinatorResult(
+        ExpectedGenericGroupResult expectedResult,
+        CoordinatorResult<Void, Record> result
+    ) {
+        if (expectedResult == null) {
+            assertEquals(EMPTY_RESULT, result);
+        } else {
+            assertEquals(expectedResult.records, result.records());
+            if (expectedResult.mockError == Errors.NONE) {
+                result.appendFuture().complete(null);
+            } else {
+                result.appendFuture().completeExceptionally(expectedResult.mockError.exception());
+            }

Review Comment:
   This is really surprising. `verify*` suggests that this method only verifies something but it also has side effects. I think that this should rather be done in the context like I did for the new 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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java:
##########
@@ -271,4 +289,110 @@ public void testOnResignation() {
             10
         );
     }
+
+    @Test
+    public void testJoinGroup() {
+        CoordinatorRuntime<ReplicatedGroupCoordinator, Record> runtime = mockRuntime();
+        GroupCoordinatorService service = new GroupCoordinatorService(
+            new LogContext(),
+            createConfig(),
+            runtime
+        );
+
+        JoinGroupRequestData request = new JoinGroupRequestData()
+            .setGroupId("foo");
+
+        service.startup(() -> 1);
+
+        when(runtime.scheduleWriteOperation(
+            ArgumentMatchers.eq("generic-group-join"),
+            ArgumentMatchers.eq(new TopicPartition("__consumer_offsets", 0)),
+            ArgumentMatchers.any()
+        )).thenReturn(CompletableFuture.completedFuture(
+            new JoinGroupResponseData()
+        ));
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = service.joinGroup(
+            requestContext(ApiKeys.JOIN_GROUP),
+            request,
+            BufferSupplier.NO_CACHING
+        );
+
+        assertFalse(responseFuture.isDone());

Review Comment:
   A successful join group request will store the response future into the member's `awaitingJoinFuture` (could also complete if the join phase completes)



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {

Review Comment:
   the syntax requires to only access final (temporary) variables. setting existing variables to new values require them to be atomic.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );

Review Comment:
   addressed in above comment.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {

Review Comment:
   the syntax requires to only access final (temporary) variables. setting existing variables to new values require them to be atomic.



-- 
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 #13870: KAFKA-14500; [5/N] Implement JoinGroup protocol in new GroupCoordinator

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


##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java:
##########
@@ -2377,156 +2606,2165 @@ public void testOnNewMetadataImage() {
 
         // Verify the groups.
         Arrays.asList("group1", "group2", "group3", "group4").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         Arrays.asList("group5").forEach(groupId -> {
-            ConsumerGroup group = context.groupMetadataManager.getOrMaybeCreateConsumerGroup(groupId, false);
+            ConsumerGroup group = context.groupMetadataManager
+                .getOrMaybeCreateConsumerGroup(groupId, false);
             assertFalse(group.hasMetadataExpired(context.time.milliseconds()));
         });
 
         // Verify image.
         assertEquals(image, context.groupMetadataManager.image());
     }
 
-    private <T> void assertUnorderedListEquals(
-        List<T> expected,
-        List<T> actual
-    ) {
-        assertEquals(new HashSet<>(expected), new HashSet<>(actual));
-    }
+    @Test
+    public void testJoinGroupShouldReceiveErrorIfGroupOverMaxSize() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(10)
+            .build();
 
-    private void assertResponseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (!responseEquals(expected, actual)) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .withReason("exceed max group size")
+            .build();
+
+        for (int i = 0; i < 10; i++) {
+            CompletableFuture<JoinGroupResponseData> responseFuture;
+            if (i == 0) {
+                responseFuture = context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                );
+            } else {
+                responseFuture = context.sendGenericGroupJoin(request);
+            }
+            assertFalse(responseFuture.isDone());
         }
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.GROUP_MAX_SIZE_REACHED.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
     }
 
-    private boolean responseEquals(
-        ConsumerGroupHeartbeatResponseData expected,
-        ConsumerGroupHeartbeatResponseData actual
-    ) {
-        if (expected.throttleTimeMs() != actual.throttleTimeMs()) return false;
-        if (expected.errorCode() != actual.errorCode()) return false;
-        if (!Objects.equals(expected.errorMessage(), actual.errorMessage())) return false;
-        if (!Objects.equals(expected.memberId(), actual.memberId())) return false;
-        if (expected.memberEpoch() != actual.memberEpoch()) return false;
-        if (expected.shouldComputeAssignment() != actual.shouldComputeAssignment()) return false;
-        if (expected.heartbeatIntervalMs() != actual.heartbeatIntervalMs()) return false;
-        // Unordered comparison of the assignments.
-        return responseAssignmentEquals(expected.assignment(), actual.assignment());
-    }
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndRequiredKnownMember() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-    private boolean responseAssignmentEquals(
-        ConsumerGroupHeartbeatResponseData.Assignment expected,
-        ConsumerGroupHeartbeatResponseData.Assignment actual
-    ) {
-        if (expected == actual) return true;
-        if (expected == null) return false;
-        if (actual == null) return false;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (!Objects.equals(fromAssignment(expected.pendingTopicPartitions()), fromAssignment(actual.pendingTopicPartitions())))
-            return false;
+        // First round of join requests. Generate member ids. All requests will be accepted
+        // as the group is still Empty.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-        return Objects.equals(fromAssignment(expected.assignedTopicPartitions()), fromAssignment(actual.assignedTopicPartitions()));
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+        assertEquals(0, group.size());
+        assertTrue(group.isInState(EMPTY));
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // the join group phase will complete.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numPendingJoinMembers());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
     }
 
-    private Map<Uuid, Set<Integer>> fromAssignment(
-        List<ConsumerGroupHeartbeatResponseData.TopicPartitions> assignment
-    ) {
-        if (assignment == null) return null;
+    @Test
+    public void testDynamicMembersJoinGroupWithMaxSizeAndNotRequiredKnownMember() {
+        boolean requiredKnownMemberId = false;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. This will trigger a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        responseFutures.add(context.sendGenericGroupJoin(
+            request,
+            requiredKnownMemberId,
+            true,
+            new ExpectedGenericGroupResult(Errors.NONE, true)
+        ));
+        for (int i = 0; i < groupMaxSize; i++) {
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members that were accepted can rejoin while others are rejected in CompletingRebalance state.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        verifyGenericGroupJoinResponses(responseFutures, 10, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertRecordsEquals(
-        List<Record> expectedRecords,
-        List<Record> actualRecords
-    ) {
-        try {
-            assertEquals(expectedRecords.size(), actualRecords.size());
+    @Test
+    public void testStaticMembersJoinGroupWithMaxSize() {
+        int groupMaxSize = 10;
 
-            for (int i = 0; i < expectedRecords.size(); i++) {
-                Record expectedRecord = expectedRecords.get(i);
-                Record actualRecord = actualRecords.get(i);
-                assertRecordEquals(expectedRecord, actualRecord);
+        List<String> groupInstanceIds = IntStream.range(0, groupMaxSize + 1)
+            .mapToObj(i -> "instance-id-" + i)
+            .collect(Collectors.toList());
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
+
+        // First round of join requests. The group metadata manager will prepare a rebalance.
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            String instanceId = groupInstanceIds.get(i);
+            request = request.setGroupInstanceId(instanceId);
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    false,
+                    true,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request));
             }
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expectedRecords)
-                .actual(actualRecords)
-                .buildAndThrow();
         }
-    }
 
-    private void assertRecordEquals(
-        Record expected,
-        Record actual
-    ) {
-        try {
-            assertApiMessageAndVersionEquals(expected.key(), actual.key());
-            assertApiMessageAndVersionEquals(expected.value(), actual.value());
-        } catch (AssertionFailedError e) {
-            assertionFailure()
-                .expected(expected)
-                .actual(actual)
-                .buildAndThrow();
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        // Members which were accepted can rejoin, others are rejected, while
+        // completing rebalance
+        responseFutures = new ArrayList<>();
+        for (int i = 0; i < groupMaxSize; i++) {
+            String memberId = memberIds.get(i);
+            String instanceId = groupInstanceIds.get(i);
+
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withGroupInstanceId(instanceId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
         }
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
 
-    private void assertApiMessageAndVersionEquals(
-        ApiMessageAndVersion expected,
-        ApiMessageAndVersion actual
-    ) {
-        if (expected == actual) return;
+    @Test
+    public void testDynamicMembersCanRejoinGroupWithMaxSizeWhileRebalancing() {
+        boolean requiredKnownMemberId = true;
+        int groupMaxSize = 10;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
 
-        assertEquals(expected.version(), actual.version());
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withDefaultProtocolTypeAndProtocols()
+            .build();
 
-        if (actual.message() instanceof ConsumerGroupCurrentMemberAssignmentValue) {
-            // The order of the topics stored in ConsumerGroupCurrentMemberAssignmentValue is not
-            // always guaranteed. Therefore, we need a special comparator.
-            ConsumerGroupCurrentMemberAssignmentValue expectedValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) expected.message();
-            ConsumerGroupCurrentMemberAssignmentValue actualValue =
-                (ConsumerGroupCurrentMemberAssignmentValue) actual.message();
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        // First round of join requests. Generate member ids.
+        for (int i = 0; i < groupMaxSize + 1; i++) {
+            if (i == 0) {
+                responseFutures.add(context.sendGenericGroupJoin(
+                    request,
+                    requiredKnownMemberId,
+                    false,
+                    new ExpectedGenericGroupResult(Errors.NONE, true)
+                ));
+            } else {
+                responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+            }
+        }
 
-            assertEquals(expectedValue.memberEpoch(), actualValue.memberEpoch());
-            assertEquals(expectedValue.previousMemberEpoch(), actualValue.previousMemberEpoch());
-            assertEquals(expectedValue.targetMemberEpoch(), actualValue.targetMemberEpoch());
-            assertEquals(expectedValue.error(), actualValue.error());
-            assertEquals(expectedValue.metadataVersion(), actualValue.metadataVersion());
-            assertEquals(expectedValue.metadataBytes(), actualValue.metadataBytes());
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            false
+        );
 
-            // We transform those to Maps before comparing them.
-            assertEquals(fromTopicPartitions(expectedValue.assignedPartitions()),
-                fromTopicPartitions(actualValue.assignedPartitions()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingRevocation()),
-                fromTopicPartitions(actualValue.partitionsPendingRevocation()));
-            assertEquals(fromTopicPartitions(expectedValue.partitionsPendingAssignment()),
-                fromTopicPartitions(actualValue.partitionsPendingAssignment()));
-        } else {
-            assertEquals(expected.message(), actual.message());
+        assertEquals(0, group.size());
+        assertEquals(groupMaxSize + 1, group.numPendingJoinMembers());
+        assertTrue(group.isInState(EMPTY));
+
+        List<String> memberIds = verifyGenericGroupJoinResponses(responseFutures, 0, Errors.MEMBER_ID_REQUIRED);
+        assertEquals(groupMaxSize + 1, memberIds.size());
+
+        // Second round of join requests with the generated member ids.
+        // One of them will fail, reaching group max size.
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            responseFutures.add(context.sendGenericGroupJoin(request.setMemberId(memberId), requiredKnownMemberId));
         }
-    }
 
-    private Map<Uuid, Set<Integer>> fromTopicPartitions(
-        List<ConsumerGroupCurrentMemberAssignmentValue.TopicPartitions> assignment
-    ) {
-        Map<Uuid, Set<Integer>> assignmentMap = new HashMap<>();
-        assignment.forEach(topicPartitions -> {
-            assignmentMap.put(topicPartitions.topicId(), new HashSet<>(topicPartitions.partitions()));
-        });
-        return assignmentMap;
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Members can rejoin while rebalancing
+        responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request, requiredKnownMemberId));
+        }
+
+        // Advance clock by group initial rebalance delay to complete first inital delayed join.
+        // This will extend the initial rebalance as new members have joined.
+        context.timer.advanceClock(50);
+        // Advance clock by group initial rebalance delay to complete second inital delayed join.
+        // Since there are no new members that joined since the previous delayed join,
+        // we will complete the rebalance.
+        context.timer.advanceClock(50);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(0, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
     }
+
+    @Test
+    public void testLastJoiningMembersAreKickedOutWhenRejoiningGroupWithMaxSize() {
+        int groupMaxSize = 10;
+
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSize(groupMaxSize)
+            .withGenericGroupInitialRebalanceDelayMs(50)
+            .build();
+
+        // Create a group and add members that exceed the group max size.
+        GenericGroup group = context.groupMetadataManager.getOrMaybeCreateGenericGroup(
+            "group-id",
+            true
+        );
+
+        List<String> memberIds = IntStream.range(0, groupMaxSize + 2)
+            .mapToObj(i -> group.generateMemberId("client-id", Optional.empty()))
+            .collect(Collectors.toList());
+
+        memberIds.forEach(memberId -> {
+            JoinGroupRequestProtocolCollection protocols = new JoinGroupRequestProtocolCollection();
+            protocols.add(new JoinGroupRequestProtocol()
+                .setName("range")
+                .setMetadata(new byte[0]));
+
+            group.add(
+                new GenericGroupMember(
+                    memberId,
+                    Optional.empty(),
+                    "client-id",
+                    "client-host",
+                    10000,
+                    5000,
+                    "consumer",
+                    protocols
+                )
+            );
+        });
+
+        context.groupMetadataManager.prepareRebalance(group, "test");
+
+        List<CompletableFuture<JoinGroupResponseData>> responseFutures = new ArrayList<>();
+        for (String memberId : memberIds) {
+            JoinGroupRequestData request = new JoinGroupRequestBuilder()
+                .withGroupId("group-id")
+                .withMemberId(memberId)
+                .withDefaultProtocolTypeAndProtocols()
+                .withRebalanceTimeoutMs(10000)
+                .build();
+
+            responseFutures.add(context.sendGenericGroupJoin(request));
+        }
+
+        assertEquals(groupMaxSize, group.size());
+        assertEquals(groupMaxSize, group.numAwaitingJoinResponse());
+        assertTrue(group.isInState(PREPARING_REBALANCE));
+
+        // Advance clock by rebalance timeout to complete join phase.
+        context.timer.advanceClock(10000);
+
+        verifyGenericGroupJoinResponses(responseFutures, groupMaxSize, Errors.GROUP_MAX_SIZE_REACHED);
+
+        assertEquals(groupMaxSize, group.size());
+        assertTrue(group.isInState(COMPLETING_REBALANCE));
+
+        memberIds.subList(groupMaxSize, groupMaxSize + 2)
+            .forEach(memberId -> assertFalse(group.hasMemberId(memberId)));
+
+        memberIds.subList(0, groupMaxSize)
+            .forEach(memberId -> assertTrue(group.hasMemberId(memberId)));
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooSmall() throws Exception {
+        int minSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMinSessionTimeoutMs(minSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(minSessionTimeout - 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupSessionTimeoutTooLarge() throws Exception {
+        int maxSessionTimeout = 50;
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .withGenericGroupMaxSessionTimeoutMs(maxSessionTimeout)
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId(UNKNOWN_MEMBER_ID)
+            .withSessionTimeoutMs(maxSessionTimeout + 1)
+            .build();
+
+        CompletableFuture<JoinGroupResponseData> responseFuture = context.sendGenericGroupJoin(request);
+        assertTrue(responseFuture.isDone());
+        assertEquals(Errors.INVALID_SESSION_TIMEOUT.code(), responseFuture.get(5, TimeUnit.SECONDS).errorCode());
+    }
+
+    @Test
+    public void testJoinGroupUnknownConsumerNewGroup() throws Exception {
+        GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
+            .build();
+
+        JoinGroupRequestData request = new JoinGroupRequestBuilder()
+            .withGroupId("group-id")
+            .withMemberId("member-id")
+            .build();
+
+        JoinGroupResponseData response = context.joinGenericGroupAsDynamicMember(request);
+
+        assertEquals(Errors.GROUP_ID_NOT_FOUND.code(), response.errorCode());

Review Comment:
   i was following the new protocol as it made more sense but i have changed to reflect the old behavior.



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