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


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

Review Comment:
   The `revoke()` and `assign()` methods return a `ReconciliationResult` to let 
the caller know the operation completed. In that case, the caller would work 
with the `HeartbeatRequestManager` so that it includes the updated partition 
set in its next 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-unsubscr...@kafka.apache.org

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

Reply via email to