jeffkbkim commented on code in PR #14481:
URL: https://github.com/apache/kafka/pull/14481#discussion_r1400045359


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilder.java:
##########
@@ -14,17 +14,891 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.kafka.coordinator.group.assignor;
 
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.server.common.TopicIdPartition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.stream.Collectors;
+
+/**
+ * The general uniform assignment builder is used to generate the target 
assignment for a consumer group with
+ * at least one of its members subscribed to a different set of topics.
+ *
+ * Assignments are done according to the following principles:
+ *
+ * <li> Balance:          Ensure partitions are distributed equally among all 
members.
+ *                        The difference in assignments sizes between any two 
members
+ *                        should not exceed one partition. </li>
+ * <li> Rack Matching:    When feasible, aim to assign partitions to members
+ *                        located on the same rack thus avoiding cross-zone 
traffic. </li>
+ * <li> Stickiness:       Minimize partition movements among members by 
retaining
+ *                        as much of the existing assignment as possible. </li>
+ *
+ * This assignment builder prioritizes the above properties in the following 
order:
+ *      Balance > Rack Matching > Stickiness.
+ */
 public class GeneralUniformAssignmentBuilder extends 
AbstractUniformAssignmentBuilder {
     private static final Logger LOG = 
LoggerFactory.getLogger(GeneralUniformAssignmentBuilder.class);
 
+    /**
+     * The member metadata obtained from the assignment specification.
+     */
+    private final Map<String, AssignmentMemberSpec> members;
+
+    /**
+     * The topic and partition metadata describer.
+     */
+    private final SubscribedTopicDescriber subscribedTopicDescriber;
+
+    /**
+     * The list of all the topic Ids that the consumer group is subscribed to.
+     */
+    private final Set<Uuid> subscriptionIds;
+
+    /**
+     * Rack information.
+     */
+    private final RackInfo rackInfo;
+
+    /**
+     * List of subscribed members for each topic.
+     */
+    private final Map<Uuid, List<String>> membersPerTopic;
+
+    /**
+     * The partitions that still need to be assigned.
+     */
+    private final Set<TopicIdPartition> unassignedPartitions;
+
+    /**
+     * All the partitions that have been retained from the existing assignment.
+     */
+    private final Set<TopicIdPartition> assignedStickyPartitions;
+
+    /**
+     * Manages assignments to members based on their current assignment size 
and maximum allowed assignment size.
+     */
+    private final AssignmentManager assignmentManager;
+
+    /**
+     * List of all the members sorted by their respective assignment sizes.
+     */
+    private final TreeSet<String> sortedMembersByAssignmentSize;
+
+    /**
+     * Tracks the owner of each partition in the existing assignment on the 
client.
+     *
+     * Only populated when rack aware strategy is used.
+     * Contains partitions that weren't retained due to a rack mismatch.
+     */
+    private final Map<TopicIdPartition, String> currentPartitionOwners;
+
+    /**
+     * Tracks the owner of each partition in the target assignment.
+     */
+    private final Map<TopicIdPartition, String> 
partitionOwnerInTargetAssignment;
+
+    /**
+     * Handles all operations related to partition movements during a 
reassignment for balancing the target assignment.
+     */
+    private PartitionMovements partitionMovements;
+
+    /**
+     * The new assignment that will be returned.
+     */
+    private final Map<String, MemberAssignment> targetAssignment;
+
+    public GeneralUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+        this.members = assignmentSpec.members();
+        this.subscribedTopicDescriber = subscribedTopicDescriber;
+        this.subscriptionIds = assignmentSpec.members().values().stream()
+            .flatMap(memberSpec -> memberSpec.subscribedTopicIds().stream())
+            .peek(topicId -> {
+                int partitionCount = 
subscribedTopicDescriber.numPartitions(topicId);
+                if (partitionCount == -1) {
+                    throw new PartitionAssignorException(
+                        "Members are subscribed to topic " + topicId + " which 
doesn't exist in the topic metadata."
+                    );
+                }
+            })
+            .collect(Collectors.toSet());
+        this.rackInfo = new RackInfo(assignmentSpec, subscribedTopicDescriber, 
subscriptionIds);
+        this.membersPerTopic = new HashMap<>();
+        members.forEach((memberId, memberMetadata) -> {
+            Collection<Uuid> topics = memberMetadata.subscribedTopicIds();
+            topics.forEach(topicId ->
+                membersPerTopic.computeIfAbsent(topicId, k -> new 
ArrayList<>()).add(memberId)
+            );
+        });
+        this.unassignedPartitions = new 
HashSet<>(topicIdPartitions(subscriptionIds, subscribedTopicDescriber));
+        this.assignedStickyPartitions = new HashSet<>();
+        this.assignmentManager = new AssignmentManager();
+        this.sortedMembersByAssignmentSize = 
assignmentManager.getSortedMembersByAssignmentSize(members.keySet());
+        this.currentPartitionOwners = new HashMap<>();
+        this.partitionOwnerInTargetAssignment = new HashMap<>();
+        this.targetAssignment = new HashMap<>();
+    }
+
+    /**
+     * Here's the step-by-step breakdown of the assignment process:
+     *
+     * <li> Retain partitions from the existing assignments a.k.a sticky 
partitions. </li>
+     *      <ul><li> If a partition's rack mismatches with its owner, track it 
for future use. </li></ul>
+     * <li> If rack aware strategy is possible, allocate unassigned partitions 
to members in the same rack. </li>
+     * <li> Allocate all the remaining unassigned partitions to the members in 
a balanced manner. If possible, allocate
+     *      the partition back to it's existing owner in case it was not 
retained earlier due to a rack mismatch. </li>
+     * <li> Iterate through the assignment until it is balanced. </li>
+     */
     @Override
     protected GroupAssignment buildAssignment() {
-        return null;
+        if (subscriptionIds.isEmpty()) {
+            LOG.info("The subscription list is empty, returning an empty 
assignment");
+            return new GroupAssignment(Collections.emptyMap());
+        }
+
+        members.keySet().forEach(memberId -> targetAssignment.put(memberId, 
new MemberAssignment(new HashMap<>())));
+        partitionMovements = new PartitionMovements();
+
+        // When rack awareness is enabled, only sticky partitions with 
matching rack are retained.
+        // Otherwise, all existing partitions are retained until max 
assignment size.
+        assignStickyPartitions();
+
+        if (rackInfo.useRackStrategy) rackAwarePartitionAssignment();
+        unassignedPartitionsAssignment();
+
+        balance();
+
+        return new GroupAssignment(targetAssignment);
+    }
+
+    /**
+     * <li> TopicIdPartitions are sorted in descending order based on the 
value:
+     *       totalPartitions/number of subscribed members. </li>
+     * <li> If the above value is the same then topicIdPartitions are sorted in
+     *      ascending order of number of subscribers. </li>
+     * <li> If both criteria are the same, sort in ascending order of the 
partition Id.
+     *      This last criteria is for predictability of the assignments. </li>
+     *
+     * @param topicIdPartitions       The topic partitions that need to be 
sorted.
+     * @return A list of sorted topic partitions.
+     */
+    private List<TopicIdPartition> 
sortTopicIdPartitions(Collection<TopicIdPartition> topicIdPartitions) {
+        Comparator<TopicIdPartition> comparator = Comparator
+            .comparingDouble((TopicIdPartition topicIdPartition) -> {
+                int totalPartitions = 
subscribedTopicDescriber.numPartitions(topicIdPartition.topicId());
+                int totalSubscribers = 
membersPerTopic.get(topicIdPartition.topicId()).size();
+                return (double) totalPartitions / totalSubscribers;
+            })
+            .reversed()
+            .thenComparingInt(topicIdPartition -> 
membersPerTopic.get(topicIdPartition.topicId()).size())
+            .thenComparingInt(TopicIdPartition::partitionId);
+
+        return topicIdPartitions.stream()
+            .sorted(comparator)
+            .collect(Collectors.toList());
+    }
+
+    /**
+     * Gets a set of partitions that are to be retained from the existing 
assignment. This includes:
+     * <li> Partitions from topics that are still present in both the new 
subscriptions and the topic metadata. </li>
+     * <li> When using a rack-aware strategy, only partitions with member 
owners in the same rack are retained. </li>
+     * <li> Track current partition owners when there is a rack mismatch. </li>
+     */
+    private void assignStickyPartitions() {
+        members.forEach((memberId, assignmentMemberSpec) ->
+            assignmentMemberSpec.assignedPartitions().forEach((topicId, 
currentAssignment) -> {
+                if 
(assignmentMemberSpec.subscribedTopicIds().contains(topicId)) {
+                    currentAssignment.forEach(partition -> {
+                        TopicIdPartition topicIdPartition = new 
TopicIdPartition(topicId, partition);
+                        if (rackInfo.useRackStrategy && 
rackInfo.racksMismatch(memberId, topicIdPartition)) {
+                            currentPartitionOwners.put(topicIdPartition, 
memberId);
+                        } else {
+                            
assignmentManager.maybeAssignPartitionToMember(topicIdPartition, memberId);
+                            assignedStickyPartitions.add(topicIdPartition);

Review Comment:
   perhaps we can add a test to trigger one of the following?
   ```
               // If member is not subscribed to the partition's topic, return 
false without assigning.
               if 
(!members.get(memberId).subscribedTopicIds().contains(topicIdPartition.topicId()))
 {
                   return false;
               }
   
               // If the member's current assignment is already at max, return 
false without assigning.
               if (isMemberAtMaxCapacity(memberId)) {
                   return false;
               }
   ```
   from maybeAssignPartitionToMember()



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