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

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

Reply via email to