lianetm commented on code in PR #14690: URL: https://github.com/apache/kafka/pull/14690#discussion_r1395947490
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/MembershipManagerImpl.java: ########## @@ -181,63 +359,592 @@ public void updateState(ConsumerGroupHeartbeatResponseData response) { public void transitionToFenced() { resetEpoch(); transitionTo(MemberState.FENCED); + + // Release assignment + CompletableFuture<Void> callbackResult = invokeOnPartitionsLostCallback(subscriptions.assignedPartitions()); + callbackResult.whenComplete((result, error) -> { + if (error != null) { + log.error("onPartitionsLost callback invocation failed while releasing assignment" + + "after member got fenced. Member will rejoin the group anyways.", error); + } + subscriptions.assignFromSubscribed(Collections.emptySet()); + transitionToJoining(); + }); + + clearPendingAssignmentsAndLocalNamesCache(); } /** * {@inheritDoc} */ @Override - public void transitionToFailed() { - log.error("Member {} transitioned to {} state", memberId, MemberState.FAILED); - transitionTo(MemberState.FAILED); + public void transitionToFatal() { + log.error("Member {} transitioned to {} state", memberId, MemberState.FATAL); + + // Update epoch to indicate that the member is not in the group anymore, so that the + // onPartitionsLost is called to release assignment. + memberEpoch = ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH; + CompletableFuture<Void> callbackResult = invokeOnPartitionsLostCallback(subscriptions.assignedPartitions()); + callbackResult.whenComplete((result, error) -> { + if (error != null) { + log.error("onPartitionsLost callback invocation failed while releasing assignment" + + "after member failed with fatal error.", error); + } + }); + subscriptions.assignFromSubscribed(Collections.emptySet()); + clearPendingAssignmentsAndLocalNamesCache(); + transitionTo(MemberState.FATAL); + } + + /** + * {@inheritDoc} + */ + public void onSubscriptionUpdated() { + if (state == MemberState.UNSUBSCRIBED) { + transitionToJoining(); + } + // TODO: If the member is already part of the group, this should only ensure that the + // updated subscription is included in the next heartbeat request. } + /** + * Transition to the {@link MemberState#JOINING} state, indicating that the member will + * try to join the group on the next heartbeat request. This is expected to be invoked when + * the user calls the subscribe API, or when the member wants to rejoin after getting fenced. + * Visible for testing. + */ + void transitionToJoining() { + if (state == MemberState.FATAL) { + log.warn("No action taken to join the group with the updated subscription because " + + "the member is in FATAL state"); + return; + } + resetEpoch(); + transitionTo(MemberState.JOINING); + clearPendingAssignmentsAndLocalNamesCache(); + registerForMetadataUpdates(); + } + + /** + * Register to get notified when the cluster metadata is updated, via the + * {@link #onUpdate(ClusterResource)}. Register only if the manager is not register already. + */ + private void registerForMetadataUpdates() { + if (!isRegisteredForMetadataUpdates) { + this.metadata.addClusterUpdateListener(this); + isRegisteredForMetadataUpdates = true; + } + } + + /** + * {@inheritDoc} + */ @Override - public boolean shouldSendHeartbeat() { - return state() != MemberState.FAILED; + public CompletableFuture<Void> leaveGroup() { + if (state == MemberState.UNSUBSCRIBED || state == MemberState.FATAL) { + // Member is not part of the group. No-op and return completed future to avoid + // unnecessary transitions. + return CompletableFuture.completedFuture(null); + } + + if (state == MemberState.PREPARE_LEAVING || state == MemberState.LEAVING) { + // Member already leaving. No-op and return existing leave group future that will + // complete when the ongoing leave operation completes. + return leaveGroupInProgress.get(); + } + + transitionTo(MemberState.PREPARE_LEAVING); + leaveGroupInProgress = Optional.of(new CompletableFuture<>()); + + CompletableFuture<Void> callbackResult = invokeOnPartitionsRevokedOrLostToReleaseAssignment(); + callbackResult.whenComplete((result, error) -> { + // Clear the subscription, no matter if the callback execution failed or succeeded. + subscriptions.assignFromSubscribed(Collections.emptySet()); + + // Transition to ensure that a heartbeat request is sent out to effectively leave the + // group (even in the case where the member had no assignment to release or when the + // callback execution failed.) + transitionToSendingLeaveGroup(); + }); + + clearPendingAssignmentsAndLocalNamesCache(); + + // Return future to indicate that the leave group is done when the callbacks + // complete, and the heartbeat to be sent out. (Best effort to send it, without waiting + // for a response or handling timeouts) + return leaveGroupInProgress.get(); } /** - * Transition to {@link MemberState#STABLE} only if there are no target assignments left to - * reconcile. Transition to {@link MemberState#RECONCILING} otherwise. + * Release member assignment by calling the user defined callbacks for onPartitionsRevoked or + * onPartitionsLost. + * <ul> + * <li>If the member is part of the group (epoch > 0), this will invoke onPartitionsRevoked. + * This will be the case when releasing assignment because the member is intentionally + * leaving the group (after a call to unsubscribe)</li> + * + * <li>If the member is not part of the group (epoch <=0), this will invoke onPartitionsLost. + * This will be the case when releasing assignment after being fenced .</li> + * </ul> + * + * @return Future that will complete when the callback execution completes. */ - private boolean maybeTransitionToStable() { - if (!hasPendingTargetAssignment()) { - transitionTo(MemberState.STABLE); + private CompletableFuture<Void> invokeOnPartitionsRevokedOrLostToReleaseAssignment() { + SortedSet<TopicPartition> droppedPartitions = new TreeSet<>(COMPARATOR); + droppedPartitions.addAll(subscriptions.assignedPartitions()); + + CompletableFuture<Void> callbackResult; + if (droppedPartitions.isEmpty()) { + // No assignment to release + callbackResult = CompletableFuture.completedFuture(null); } else { - transitionTo(MemberState.RECONCILING); + // Release assignment + if (memberEpoch > 0) { + // Member is part of the group. Invoke onPartitionsRevoked. + callbackResult = revokePartitions(droppedPartitions); + } else { + // Member is not part of the group anymore. Invoke onPartitionsLost. + callbackResult = invokeOnPartitionsLostCallback(droppedPartitions); + } + // Remove all topic IDs and names from local cache + callbackResult.whenComplete((result, error) -> clearPendingAssignmentsAndLocalNamesCache()); } - return state.equals(MemberState.STABLE); + return callbackResult; + } + + /** + * Reset member epoch to the value required for the leave the group heartbeat request, and + * transition to the {@link MemberState#LEAVING} state so that a heartbeat + * request is sent out with it. + */ + private void transitionToSendingLeaveGroup() { + memberEpoch = ConsumerGroupHeartbeatRequest.LEAVE_GROUP_MEMBER_EPOCH; + currentAssignment = new HashSet<>(); + transitionTo(MemberState.LEAVING); + } + + /** + * {@inheritDoc} + */ + @Override + public boolean shouldHeartbeatNow() { + MemberState state = state(); + return state == MemberState.ACKNOWLEDGING || state == MemberState.LEAVING; } /** - * Take new target assignment received from the server and set it as targetAssignment to be - * processed. Following the consumer group protocol, the server won't send a new target - * member while a previous one hasn't been acknowledged by the member, so this will fail - * if a target assignment already exists. + * {@inheritDoc} + */ + @Override + public void onHeartbeatRequestSent() { + MemberState state = state(); + if (state == MemberState.ACKNOWLEDGING) { + if (allPendingAssignmentsReconciled()) { + transitionTo(MemberState.STABLE); + } else { + transitionTo(MemberState.RECONCILING); + } + } else if (state == MemberState.LEAVING) { + transitionTo(MemberState.UNSUBSCRIBED); + leaveGroupInProgress.get().complete(null); + leaveGroupInProgress = Optional.empty(); + } + } + + /** + * @return True if there are no assignments waiting to be resolved from metadata or reconciled. + */ + private boolean allPendingAssignmentsReconciled() { + return assignmentUnresolved.isEmpty() && assignmentReadyToReconcile.isEmpty(); + } + + @Override + public boolean shouldSkipHeartbeat() { + MemberState state = state(); + return state == MemberState.UNSUBSCRIBED || state == MemberState.FATAL; + } + + /** + * Reconcile the assignment that has been received from the server and for which topic names + * are resolved, kept in the {@link #assignmentReadyToReconcile}. This will commit if needed, + * trigger the callbacks and update the subscription state. Note that only one reconciliation + * can be in progress at a time. If there is already another one in progress when this is + * triggered, it will be no-op, and the assignment will be reconciled on the next + * reconciliation loop. + */ + boolean reconcile() { + // Make copy of the assignment to reconcile as it could change as new assignments or metadata updates are received + SortedSet<TopicPartition> assignedPartitions = new TreeSet<>(COMPARATOR); + assignedPartitions.addAll(assignmentReadyToReconcile); + + SortedSet<TopicPartition> ownedPartitions = new TreeSet<>(COMPARATOR); + ownedPartitions.addAll(subscriptions.assignedPartitions()); + + boolean sameAssignmentReceived = assignedPartitions.equals(ownedPartitions); + + if (reconciliationInProgress || sameAssignmentReceived) { + String reason; + if (reconciliationInProgress) { + reason = "Another reconciliation is already in progress. Assignment " + + assignmentReadyToReconcile + " will be handled in the next reconciliation loop."; + } else { + reason = "Target assignment ready to reconcile is equals to the member current assignment."; + } + log.debug("Ignoring reconciliation attempt. " + reason); + return false; + } + + markReconciliationInProgress(); + + // Partitions to assign (not previously owned) + SortedSet<TopicPartition> addedPartitions = new TreeSet<>(COMPARATOR); + addedPartitions.addAll(assignedPartitions); + addedPartitions.removeAll(ownedPartitions); + + // Partitions to revoke + SortedSet<TopicPartition> revokedPartitions = new TreeSet<>(COMPARATOR); + revokedPartitions.addAll(ownedPartitions); + revokedPartitions.removeAll(assignedPartitions); + + log.info("Updating assignment with\n" + + "\tAssigned partitions: {}\n" + + "\tCurrent owned partitions: {}\n" + + "\tAdded partitions (assigned - owned): {}\n" + + "\tRevoked partitions (owned - assigned): {}\n", + assignedPartitions, + ownedPartitions, + addedPartitions, + revokedPartitions + ); + + CompletableFuture<Void> revocationResult; + if (!revokedPartitions.isEmpty()) { + revocationResult = revokePartitions(revokedPartitions); + } else { + revocationResult = CompletableFuture.completedFuture(null); + // Reschedule the auto commit starting from now (new assignment received without any + // revocation). + commitRequestManager.resetAutoCommitTimer(); + } + + // Future that will complete when the full reconciliation process completes (revocation + // and assignment, executed sequentially) + CompletableFuture<Void> reconciliationResult = + revocationResult.thenCompose(__ -> { + boolean memberHasRejoined = !Objects.equals(memberEpochOnReconciliationStart, + memberEpoch); + if (state == MemberState.RECONCILING && !memberHasRejoined) { + // Apply assignment + CompletableFuture<Void> assignResult = assignPartitions(assignedPartitions, addedPartitions); + + // Clear topic names cache only for topics that are not in the subscription anymore + for (TopicPartition tp : revokedPartitions) { + if (!subscriptions.subscription().contains(tp.topic())) { + assignedTopicNamesCache.values().remove(tp.topic()); + } + } + return assignResult; + } else { + String reason; + if (state != MemberState.RECONCILING) { + reason = "The member already transitioned out of the reconciling " + + "state into " + state; + } else { + reason = "The member has re-joined the group."; + } + // Revocation callback completed but the reconciled assignment should not + // be applied (not relevant anymore). This could be because the member + // is not in the RECONCILING state anymore (fenced, failed, unsubscribed), + // or because it has already re-joined the group. + CompletableFuture<Void> res = new CompletableFuture<>(); + res.completeExceptionally(new KafkaException("Interrupting reconciliation" + + " after revocation. " + reason)); + return res; + } + }); + + reconciliationResult.whenComplete((result, error) -> { + markReconciliationCompleted(); + if (error != null) { + // Leaving member in RECONCILING state after callbacks fail. The member + // won't send the ack, and the expectation is that the broker will kick the + // member out of the group after the rebalance timeout expires, leading to a + // RECONCILING -> FENCED transition. + log.error("Reconciliation failed.", error); + } else { + if (state == MemberState.RECONCILING) { Review Comment: Yes, I think we need it too. Added. -- 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