lianetm commented on code in PR #14357:
URL: https://github.com/apache/kafka/pull/14357#discussion_r1333122605


##########
clients/src/main/java/org/apache/kafka/clients/consumer/internals/MemberAssignmentReconciler.java:
##########
@@ -0,0 +1,291 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.clients.consumer.internals;
+
+import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
+import 
org.apache.kafka.clients.consumer.internals.events.AssignPartitionsEvent;
+import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent;
+import 
org.apache.kafka.clients.consumer.internals.events.RebalanceCallbackEvent;
+import 
org.apache.kafka.clients.consumer.internals.events.RevokePartitionsEvent;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
+import 
org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData.Assignment;
+import 
org.apache.kafka.common.message.ConsumerGroupHeartbeatResponseData.TopicPartitions;
+import 
org.apache.kafka.clients.consumer.internals.Utils.TopicPartitionComparator;
+import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.common.utils.Timer;
+import org.slf4j.Logger;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.BlockingQueue;
+import java.util.stream.Collectors;
+
+/**
+ * {@code MemberAssignmentReconciler} works with {@link MembershipManager} to 
a) determine, and b) modify the
+ * current set of assigned {@link TopicPartition partitions} via {@link 
SubscriptionState}. Reconciliation is a
+ * two-part process, the first being a revocation of partitions, followed by 
assignment of partitions. Each of the two
+ * steps may result in one of the following:
+ *
+ * <ul>
+ *     <li>
+ *         {@link ReconciliationResult#NO_CHANGE}: no changes were made to the 
set of partitions.
+ *     </li>
+ *     <li>
+ *         {@link ReconciliationResult#IN_PROGRESS}: changes to the assignment 
have started. In practice this means
+ *         that the appropriate {@link ConsumerRebalanceListener} callback 
method is being invoked.
+ *     </li>
+ *     <li>
+ *         {@link ReconciliationResult#COMPLETED}: the {@link 
ConsumerRebalanceListener} callback method was made and
+ *         the changes were applied locally.
+ *     </li>
+ *     <li>
+ *         {@link ReconciliationResult#EXPIRED}: something happened to cause 
the operation to modify the assigned set
+ *         of partitions. This could be caused by a {@link 
ConsumerRebalanceListener} callback method that takes too
+ *         long to execute, interruption with the consumer group coordinator, 
or other reasons.
+ *     </li>
+ * </ul>
+ *
+ * The comparison against the {@link SubscriptionState#assignedPartitions() 
current set of assigned partitions} and
+ * the {@link Assignment#assignedTopicPartitions() target set of assigned 
partitions} is performed by essentially
+ * <em>flattening</em> the respective entries into two sets of {@link 
org.apache.kafka.common.TopicPartition partitons}
+ * which are then compared using basic {@link Set} comparisons.
+ */
+public class MemberAssignmentReconciler {
+
+    /**
+     * The result of the {@link #revoke(Optional, Timer)} or {@link 
#assign(Optional, Timer)} methods being invoked.
+     */
+    enum ReconciliationResult {
+        NO_CHANGE,
+        IN_PROGRESS,
+        COMPLETED,
+        EXPIRED
+    }
+
+    // Ugly little handler enum for making logging less verbose.
+    private enum Operation {
+
+        REVOKE("revoke", "revoked"), ASSIGN("assign", "assigned");
+
+        private final String verbPastTense;
+
+        private final String methodName;
+
+        Operation(String verbPresentTense, String verbPastTense) {
+            this.verbPastTense = verbPastTense;
+            this.methodName = String.format("%s.onPartitions%s()", 
ConsumerRebalanceListener.class.getSimpleName(), verbPresentTense.substring(0, 
1).toUpperCase(Locale.ROOT) + 
verbPresentTense.substring(1).toLowerCase(Locale.ROOT));
+        }
+    }
+
+    private final Logger log;
+    private final SubscriptionState subscriptions;
+    private final ConsumerMetadata metadata;
+    private final BlockingQueue<BackgroundEvent> backgroundEventQueue;
+    private Optional<RebalanceCallbackEvent> inflightCallback;
+
+    public MemberAssignmentReconciler(LogContext logContext,
+                                      SubscriptionState subscriptions,
+                                      ConsumerMetadata metadata,
+                                      BlockingQueue<BackgroundEvent> 
backgroundEventQueue) {
+        this.log = logContext.logger(getClass());
+        this.subscriptions = subscriptions;
+        this.metadata = metadata;
+        this.backgroundEventQueue = backgroundEventQueue;
+    }
+
+    /**
+     * Perform the revocation process, if necessary, depending on the given 
{@link Assignment target assignment}. If the
+     * {@link SubscriptionState#assignedPartitions() current set of assigned 
partitions} includes entries that are
+     * <em>not</em> in the target assignment, these will be considered for 
revocation. If there is already a
+     * reconciliation in progress (revocation or assignment), this method will 
return without performing any
+     * revocation.
+     *
+     * @param assignment Target {@link Assignment}
+     * @param timer Timer with which to ensure that the operation perform 
within a specified threshold
+     * @return {@link ReconciliationResult}
+     */
+    ReconciliationResult revoke(Optional<Assignment> assignment, Timer timer) {
+        // For revoke, filter out any partitions in our "current" assignment 
that aren't in the "target" assignment.
+        PartitionDiffer partitionDiffer = (current, target) -> current
+                .stream()
+                .filter(tp -> !target.contains(tp))
+                .collect(Collectors.toSet());
+
+        // For revocation, we modify the "current" assignment by removing all 
entries that were not present in the
+        // "target" assignment.
+        AssignmentModifier modifier = Set::removeAll;
+
+        return reconcile(assignment,
+                timer,
+                Operation.REVOKE,
+                partitionDiffer,
+                RevokePartitionsEvent::new,
+                modifier);
+    }
+
+    ReconciliationResult assign(Optional<Assignment> assignment, Timer timer) {
+        PartitionDiffer partitionDiffer = (current, target) -> target
+                .stream()
+                .filter(tp -> !current.contains(tp))
+                .collect(Collectors.toSet());
+
+        // For assignment, we modify the "current" assignment by adding all 
entries that were present in the
+        // "target" assignment but not in the "current" assignment.
+        AssignmentModifier modifier = Set::addAll;
+
+        return reconcile(assignment,
+                timer,
+                Operation.ASSIGN,
+                partitionDiffer,
+                AssignPartitionsEvent::new,
+                modifier);
+    }
+
+    private ReconciliationResult reconcile(Optional<Assignment> assignment,
+                                           Timer timer,
+                                           Operation operation,
+                                           PartitionDiffer partitionDiffer,
+                                           EventGenerator eventGenerator,
+                                           AssignmentModifier 
assignmentModifier) {
+        // Check for any outstanding operations first. If a conclusive result 
has already been reached, return that
+        // before processing any further.
+        Optional<ReconciliationResult> inflightStatus = 
checkInflightStatus(timer);
+
+        if (inflightStatus.isPresent())
+            return inflightStatus.get();
+
+        // "diff" the two sets of partitions: our "current" assignment and the 
"target" assignment. The result is
+        // sorted primarily so when the partitions show up in the logs, it's 
easier for us humans to understand.
+        Set<TopicPartition> diffPartitions = 
sortPartitions(partitionDiffer.diff(subscriptions.assignedPartitions(),
+                targetPartitions(metadata, assignment)));
+
+        // TODO: should this be here? Do we expect the above to do anything 
besides move things around in memory?
+        timer.update();
+
+        if (diffPartitions.isEmpty()) {
+            log.debug("No partitions were {} in the new assignment", 
operation.verbPastTense);
+            return ReconciliationResult.NO_CHANGE;
+        }
+
+        // Set up our callback invocation. We don't block here waiting on it 
its completion, though.
+        log.debug("Preparing to invoke {} with the following partitions: {}", 
operation.methodName, diffPartitions);
+        RebalanceCallbackEvent event = 
eventGenerator.generate(Collections.unmodifiableList(new 
ArrayList<>(diffPartitions)));
+        inflightCallback = Optional.of(event);
+
+        // Enqueue it in our background->application shared queue. This should 
be invoked in the Consumer.poll() method.
+        backgroundEventQueue.add(event);
+
+        // When the core event is completed in the application via 
Consumer.poll(), we want to invoke our code to
+        // "commit" the change.
+        event.future().whenComplete((result, error) -> {
+            if (error != null) {
+                log.warn("An error occurred when invoking {} with the 
following partitions: {}", operation.methodName, diffPartitions, error);
+                // TODO: should we proceed or abort?

Review Comment:
   The old consumer completely fails (abort the poll loop) with a 
KafkaException when there is a failure in any of the callbacks (exception 
starting 
[here](https://github.com/apache/kafka/blob/9e5ca8416de6011fa8c0d63430e9b89fe751ee2b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java#L485))



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