dajac commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1315500509


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilder.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class GeneralUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+    private static final Logger log = 
LoggerFactory.getLogger(GeneralUniformAssignmentBuilder.class);

Review Comment:
   nit: `log` -> `LOG` as this is a constant.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilder.java:
##########
@@ -0,0 +1,31 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class GeneralUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+    private static final Logger log = 
LoggerFactory.getLogger(GeneralUniformAssignmentBuilder.class);
+
+    @Override
+    protected GroupAssignment buildAssignment() {
+        return null;
+

Review Comment:
   nit: We can remove this empty line.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *      balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+    private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+    private final AssignmentSpec assignmentSpec;
+    private final SubscribedTopicDescriber subscribedTopicDescriber;
+    // List of topics subscribed to by all members.
+    private final Set<Uuid> subscriptionIds;
+    private final RackInfo rackInfo;
+    // Count of members to receive an extra partition beyond the minimum quota,
+    // to account for the distribution of the remaining partitions.
+    private int remainingMembersToGetAnExtraPartition;
+    // Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+    // including members eligible for an extra partition.
+    private final Map<String, Integer> potentiallyUnfilledMembers;
+    // Members mapped to the remaining number of partitions needed to meet the 
full quota.
+    // Full quota = minQuota + one extra partition (if applicable).
+    private Map<String, Integer> unfilledMembers;
+    private List<TopicIdPartition> unassignedPartitions;
+    private final Map<String, MemberAssignment> newAssignment;
+    // Tracks the current owner of each partition when using rack-aware 
strategy.
+    // Current refers to the existing assignment.
+    private final Map<TopicIdPartition, String> currentPartitionOwners;
+    // Indicates if a rack aware assignment can be done.
+    // True if racks are defined for both members and partitions.
+    boolean useRackAwareStrategy;
+
+    OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+        this.assignmentSpec = assignmentSpec;
+        this.subscribedTopicDescriber = subscribedTopicDescriber;
+        this.subscriptionIds = new 
HashSet<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+        this.rackInfo = new RackInfo(assignmentSpec, subscribedTopicDescriber, 
subscriptionIds);
+        this.potentiallyUnfilledMembers = new HashMap<>();
+        this.unfilledMembers = new HashMap<>();
+        this.newAssignment = new HashMap<>();
+        this.useRackAwareStrategy = rackInfo.useRackStrategy;

Review Comment:
   nit: Do we need to define `useRackAwareStrategy` as an attribute if we can 
get it from `rackInfo.useRackStrategy`? We could perhaps have a method instead.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *      balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+    private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+    private final AssignmentSpec assignmentSpec;
+    private final SubscribedTopicDescriber subscribedTopicDescriber;
+    // List of topics subscribed to by all members.
+    private final Set<Uuid> subscriptionIds;
+    private final RackInfo rackInfo;
+    // Count of members to receive an extra partition beyond the minimum quota,
+    // to account for the distribution of the remaining partitions.
+    private int remainingMembersToGetAnExtraPartition;
+    // Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+    // including members eligible for an extra partition.
+    private final Map<String, Integer> potentiallyUnfilledMembers;
+    // Members mapped to the remaining number of partitions needed to meet the 
full quota.
+    // Full quota = minQuota + one extra partition (if applicable).
+    private Map<String, Integer> unfilledMembers;
+    private List<TopicIdPartition> unassignedPartitions;
+    private final Map<String, MemberAssignment> newAssignment;
+    // Tracks the current owner of each partition when using rack-aware 
strategy.
+    // Current refers to the existing assignment.
+    private final Map<TopicIdPartition, String> currentPartitionOwners;
+    // Indicates if a rack aware assignment can be done.
+    // True if racks are defined for both members and partitions.
+    boolean useRackAwareStrategy;
+
+    OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+        this.assignmentSpec = assignmentSpec;
+        this.subscribedTopicDescriber = subscribedTopicDescriber;
+        this.subscriptionIds = new 
HashSet<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());

Review Comment:
   I suppose that we assume that we cannot get here if members is empty. Is 
this correct?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+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.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * <ul>
+ *     <li>
+ *         <b> Optimized Uniform Assignment Builder: </b> This strategy is 
used when all members have subscribed
+ *         to the same set of topics.
+ *     </li>
+ *     <li>
+ *         <b> General Uniform Assignment Builder: </b> This strategy is used 
when members have varied topic
+ *         subscriptions.
+ *     </li>
+ * </ul>
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+    private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+    public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+    @Override
+    public String name() {
+        return UNIFORM_ASSIGNOR_NAME;
+    }
+
+    /**
+     * Perform the group assignment given the current members and
+     * topic metadata.
+     *
+     * @param assignmentSpec                The member assignment spec.
+     * @param subscribedTopicDescriber      The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+     * @return The new assignment for the group.
+     */
+    @Override
+    public GroupAssignment assign(
+        AssignmentSpec assignmentSpec,
+        SubscribedTopicDescriber subscribedTopicDescriber
+    ) throws PartitionAssignorException {
+        AbstractAssignmentBuilder assignmentBuilder;
+        if (assignmentSpec.members().isEmpty())
+            return new GroupAssignment(Collections.emptyMap());
+
+        if (allSubscriptionsEqual(assignmentSpec.members())) {
+            log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
+                + "optimized assignment algorithm");
+            assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+        } else {
+            assignmentBuilder = new GeneralUniformAssignmentBuilder();
+            log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
+                + "general assignment algorithm");
+        }
+
+        return assignmentBuilder.buildAssignment();
+    }
+
+    /**
+     * Determines if all members are subscribed to the same list of topic IDs.
+     *
+     * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+     *                Assumes the map is non-empty.
+     * @return true if all members have the same subscription list of topic 
IDs,
+     *         false otherwise.
+     */
+    private boolean allSubscriptionsEqual(Map<String, AssignmentMemberSpec> 
members) {
+        Set<Uuid> firstSubscriptionSet = new 
HashSet<>(members.values().iterator().next().subscribedTopicIds());
+        for (AssignmentMemberSpec memberSpec : members.values()) {
+            Set<Uuid> currentSubscriptionSet = new 
HashSet<>(memberSpec.subscribedTopicIds());

Review Comment:
   Is using an HashSet necessary here? We could perhaps use 
`firstSubscriptionSet.containsAll(memberSpec.subscribedTopicIds())`. Would it 
work?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/consumer/SubscribedTopicMetadata.java:
##########
@@ -68,12 +68,17 @@ public int numPartitions(Uuid topicId) {
      * @param topicId       Uuid corresponding to the partition's topic.
      * @param partition     Partition Id within the topic.
      * @return The set of racks corresponding to the replicas of the topics 
partition.
-     *         If the topic Id does not exist, an empty set is returned
+     *         If the topic Id does not exist or no partition rack information 
is available, an empty set is returned.
      */
     @Override
     public Set<String> racksForPartition(Uuid topicId, int partition) {
         TopicMetadata topic = this.topicMetadata.get(topicId);
-        return topic == null ? Collections.emptySet() : 
topic.partitionRacks().get(partition);
+        if (topic == null) {

Review Comment:
   nit: Could we keep the previous code and use `getOrDefault` instead of 
`get`? The code would be more concise...



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java:
##########
@@ -89,16 +93,17 @@ public MemberWithRemainingAssignments(String memberId, int 
remaining) {
     private Map<Uuid, List<String>> membersPerTopic(final AssignmentSpec 
assignmentSpec, final SubscribedTopicDescriber subscribedTopicDescriber) {
         Map<Uuid, List<String>> membersPerTopic = new HashMap<>();
         Map<String, AssignmentMemberSpec> membersData = 
assignmentSpec.members();
-
+        // Only add topic Ids to the map if they are present in the topic 
metadata.
         membersData.forEach((memberId, memberMetadata) -> {
             Collection<Uuid> topics = memberMetadata.subscribedTopicIds();
             for (Uuid topicId : topics) {
                 if (subscribedTopicDescriber.numPartitions(topicId) == -1) {
-                    throw new PartitionAssignorException("Member is subscribed 
to a non-existent topic");
+                    log.warn("Members are subscribed to topic " + topicId + " 
which doesn't exist in the topic metadata.");

Review Comment:
   Shouldn't we keep what we had here?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *      balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+    private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+    private final AssignmentSpec assignmentSpec;
+    private final SubscribedTopicDescriber subscribedTopicDescriber;
+    // List of topics subscribed to by all members.
+    private final Set<Uuid> subscriptionIds;
+    private final RackInfo rackInfo;
+    // Count of members to receive an extra partition beyond the minimum quota,
+    // to account for the distribution of the remaining partitions.
+    private int remainingMembersToGetAnExtraPartition;
+    // Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+    // including members eligible for an extra partition.
+    private final Map<String, Integer> potentiallyUnfilledMembers;
+    // Members mapped to the remaining number of partitions needed to meet the 
full quota.
+    // Full quota = minQuota + one extra partition (if applicable).
+    private Map<String, Integer> unfilledMembers;
+    private List<TopicIdPartition> unassignedPartitions;
+    private final Map<String, MemberAssignment> newAssignment;
+    // Tracks the current owner of each partition when using rack-aware 
strategy.
+    // Current refers to the existing assignment.
+    private final Map<TopicIdPartition, String> currentPartitionOwners;
+    // Indicates if a rack aware assignment can be done.
+    // True if racks are defined for both members and partitions.
+    boolean useRackAwareStrategy;
+
+    OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+        this.assignmentSpec = assignmentSpec;
+        this.subscribedTopicDescriber = subscribedTopicDescriber;
+        this.subscriptionIds = new 
HashSet<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+        this.rackInfo = new RackInfo(assignmentSpec, subscribedTopicDescriber, 
subscriptionIds);
+        this.potentiallyUnfilledMembers = new HashMap<>();
+        this.unfilledMembers = new HashMap<>();
+        this.newAssignment = new HashMap<>();
+        this.useRackAwareStrategy = rackInfo.useRackStrategy;
+        // Without rack-aware strategy, tracking current owners of unassigned 
partitions is unnecessary
+        // as all sticky partitions are retained until a member meets its 
quota.
+        this.currentPartitionOwners = useRackAwareStrategy ? new HashMap<>() : 
Collections.emptyMap();
+    }
+
+    /**
+     * Here's the step-by-step breakdown of the assignment process:
+     *
+     * <li> Compute the quotas of partitions for each member based on the 
total partitions and member count.</li>
+     * <li> For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.</li>
+     * <li> If a partition's rack mismatches with its owner, track it for 
future use.</li>
+     * <li> Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.</li>
+     * <li> Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.</li>
+     * <li> Depending on members needing extra partitions, select members from 
the potentially unfilled list
+     *      and add them to the unfilled list.</li>
+     * <li> Proceed with a round-robin assignment adhering to rack awareness.
+     *      For each unassigned partition, locate the first compatible member 
from the unfilled list.</li>
+     * <li> If no rack-compatible member is found, revert to the tracked 
current owner.
+     *      If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.</li>
+     * */
+    @Override
+    protected GroupAssignment buildAssignment() throws 
PartitionAssignorException{
+        int totalPartitionsCount = 0;
+        // Removes the current topic from subscriptionIds if the topic doesn't 
exist in the topic metadata.
+        for (Uuid topicId : subscriptionIds) {
+            int partitionCount = 
subscribedTopicDescriber.numPartitions(topicId);
+            if (partitionCount == -1) {
+                IllegalStateException illegalStateException =
+                    new IllegalStateException("The topic " + topicId + " seems 
to be in an invalid state.");
+
+                throw new PartitionAssignorException(
+                    "Members are subscribed to topic " + topicId + " which 
doesn't exist in the topic metadata.",
+                    illegalStateException
+                );
+            } else {
+                totalPartitionsCount += partitionCount;
+            }
+        }
+
+        if (subscriptionIds.isEmpty()) {
+            log.info("The subscription list is empty, returning an empty 
assignment");
+            return new GroupAssignment(Collections.emptyMap());
+        }
+
+        // The minimum required quota that each member needs to meet for a 
balanced assignment.
+        // This is the same for all members.
+        int numberOfMembers = assignmentSpec.members().size();
+        final int minQuota = totalPartitionsCount / numberOfMembers;
+        remainingMembersToGetAnExtraPartition = totalPartitionsCount % 
numberOfMembers;
+
+        assignmentSpec.members().keySet().forEach(memberId ->
+            newAssignment.put(memberId, new MemberAssignment(new HashMap<>())
+        ));
+
+        Set<TopicIdPartition> allAssignedStickyPartitions = 
computeAssignedStickyPartitions(minQuota);
+        unassignedPartitions = 
computeUnassignedPartitions(allAssignedStickyPartitions);
+        unfilledMembers = computeUnfilledMembers();
+
+        if (!unassignedPartitionsCountEqualsRemainingAssignmentsCount()) {
+            throw new PartitionAssignorException("Number of available 
partitions is not equal to the total requirement");
+        }
+
+        if (useRackAwareStrategy) rackAwareRoundRobinAssignment();
+        unassignedPartitionsRoundRobinAssignment();
+
+        return new GroupAssignment(newAssignment);
+    }
+
+    /**
+     * Retrieves a set of partitions that were currently assigned to members 
and will be retained in the new assignment,
+     * by ensuring that the partitions are still relevant based on current 
topic metadata and subscriptions.
+     * If rack awareness is enabled, it ensures that a partition's rack 
matches the member's rack.
+     *
+     * <p> For each member, it:
+     * <ul>
+     *     <li> Finds the valid current assignment considering topic 
subscriptions and metadata.</li>
+     *     <li> If current assignments exist, retains up to the minimum quota 
of assignments.</li>
+     *     <li> If there are members that should get an extra partition,
+     *          assigns the next partition after the retained ones.</li>
+     *     <li> For members with assignments not exceeding the minimum quota,
+     *          it identifies them as potentially unfilled members and tracks 
the remaining quota.</li>
+     * </ul>
+     *
+     * @return A set containing all the sticky partitions that have been 
retained in the new assignment.
+     */
+    private Set<TopicIdPartition> computeAssignedStickyPartitions(Integer 
minQuota) {
+        Set<TopicIdPartition> allAssignedStickyPartitions = new HashSet<>();
+
+        assignmentSpec.members().forEach((memberId, assignmentMemberSpec) -> {
+            // Remove all the topics that aren't in the subscriptions or the 
topic metadata anymore.
+            // If rack awareness is enabled, only add partitions if the 
members rack matches the partitions rack.
+            List<TopicIdPartition> validCurrentAssignment = 
validCurrentAssignment(
+                memberId,
+                assignmentMemberSpec.assignedPartitions()
+            );
+
+            int currentAssignmentSize = validCurrentAssignment.size();
+            int remaining = minQuota - currentAssignmentSize;
+
+            if (currentAssignmentSize > 0) {
+                int retainedPartitionsCount = min(currentAssignmentSize, 
minQuota);
+                IntStream.range(0, retainedPartitionsCount).forEach(i -> {
+                    newAssignment.get(memberId)
+                        .targetPartitions()
+                        
.computeIfAbsent(validCurrentAssignment.get(i).topicId(), __ -> new HashSet<>())
+                        .add(validCurrentAssignment.get(i).partition());
+                    
allAssignedStickyPartitions.add(validCurrentAssignment.get(i));
+                });
+
+                // The extra partition is located at the last index from the 
previous step.
+                if (remaining < 0 && remainingMembersToGetAnExtraPartition > 
0) {
+                    newAssignment.get(memberId)
+                        .targetPartitions()
+                        
.computeIfAbsent(validCurrentAssignment.get(retainedPartitionsCount).topicId(), 
__ -> new HashSet<>())
+                        
.add(validCurrentAssignment.get(retainedPartitionsCount).partition());
+                    
allAssignedStickyPartitions.add(validCurrentAssignment.get(retainedPartitionsCount));
+                    remainingMembersToGetAnExtraPartition--;
+                }
+            }
+
+            if (remaining >= 0) {
+                potentiallyUnfilledMembers.put(memberId, remaining);
+            }
+
+        });
+
+        return allAssignedStickyPartitions;
+    }
+
+    /**
+     * Filters the current assignment of partitions for a given member.
+     *
+     * If a partition is assigned to a member not subscribed to its topic or
+     * if the rack-aware strategy is to be used but there is a mismatch,
+     * the partition is excluded from the valid assignment and stored for 
future consideration.
+     *
+     * @param memberId              The Id of the member whose assignment is 
being validated.
+     * @param assignedPartitions    The partitions currently assigned to the 
member.
+     *
+     * @return List of valid partitions after applying the filters.
+     */
+    private List<TopicIdPartition> validCurrentAssignment(
+        String memberId,
+        Map<Uuid, Set<Integer>> assignedPartitions
+    ) {
+        List<TopicIdPartition> validCurrentAssignmentList = new ArrayList<>();
+        assignedPartitions.forEach((topicId, currentAssignment) -> {
+            if (subscriptionIds.contains(topicId)) {
+                currentAssignment.forEach(partition -> {
+                    TopicIdPartition topicIdPartition = new 
TopicIdPartition(topicId, partition);
+                    if (useRackAwareStrategy && 
rackInfo.racksMismatch(memberId, topicIdPartition)) {
+                        currentPartitionOwners.put(topicIdPartition, memberId);
+                    } else {
+                        validCurrentAssignmentList.add(topicIdPartition);
+                    }
+                });
+            }
+            else {
+                log.debug("The topic " + topicId + " is no longer present in 
the subscribed topics list");
+            }
+        });
+
+        return validCurrentAssignmentList;
+    }
+
+    /**
+     * This method iterates over the unassigned partitions and attempts to 
allocate them
+     * to members while considering their rack affiliations.
+     */
+    private void rackAwareRoundRobinAssignment() {
+        Queue<String> roundRobinMembers = new 
LinkedList<>(unfilledMembers.keySet());
+
+        // Sort partitions in ascending order by number of potential members 
with matching racks.
+        // Partitions with no potential members aren't included in this list.
+        List<TopicIdPartition> sortedPartitions = 
rackInfo.sortPartitionsByRackMembers(unassignedPartitions);
+
+        sortedPartitions.forEach(partition -> {
+            boolean assigned = false;
+            for (int i = 0; i < roundRobinMembers.size() && !assigned; i++) {
+                String memberId = roundRobinMembers.poll();
+                Integer remainingPartitionCount = 
unfilledMembers.getOrDefault(memberId, 0);
+
+                if (remainingPartitionCount > 0 && 
!rackInfo.racksMismatch(memberId, partition)) {
+                    assignPartitionToMember(memberId, partition);
+                    assigned = true;
+                    unassignedPartitions.remove(partition);
+                }
+
+                // Only re-add to the end of the queue if it's still in the 
unfilledMembers map
+                if (unfilledMembers.containsKey(memberId)) {
+                    roundRobinMembers.add(memberId);
+                }
+            }
+        });
+    }
+
+    /**
+     * Allocates the unassigned partitions to available members.
+     *
+     * If the rack-aware strategy is enabled, partitions are attempted to be 
assigned back to their current owners first.
+     *
+     * If a partition couldn't be assigned to its current owner due to quotas 
or
+     * if the rack-aware strategy is not enabled, the partitions are allocated 
to members in a round-robin fashion.</p>

Review Comment:
   nit: We can remove `</p>`.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *      balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+    private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+    private final AssignmentSpec assignmentSpec;

Review Comment:
   Could we please add javadoc to all the attributes? We have been doing this 
for all the new code in this module so we should continue.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *      balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+    private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+    private final AssignmentSpec assignmentSpec;
+    private final SubscribedTopicDescriber subscribedTopicDescriber;
+    // List of topics subscribed to by all members.
+    private final Set<Uuid> subscriptionIds;
+    private final RackInfo rackInfo;
+    // Count of members to receive an extra partition beyond the minimum quota,
+    // to account for the distribution of the remaining partitions.
+    private int remainingMembersToGetAnExtraPartition;
+    // Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+    // including members eligible for an extra partition.
+    private final Map<String, Integer> potentiallyUnfilledMembers;
+    // Members mapped to the remaining number of partitions needed to meet the 
full quota.
+    // Full quota = minQuota + one extra partition (if applicable).
+    private Map<String, Integer> unfilledMembers;
+    private List<TopicIdPartition> unassignedPartitions;
+    private final Map<String, MemberAssignment> newAssignment;
+    // Tracks the current owner of each partition when using rack-aware 
strategy.
+    // Current refers to the existing assignment.
+    private final Map<TopicIdPartition, String> currentPartitionOwners;
+    // Indicates if a rack aware assignment can be done.
+    // True if racks are defined for both members and partitions.
+    boolean useRackAwareStrategy;
+
+    OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+        this.assignmentSpec = assignmentSpec;
+        this.subscribedTopicDescriber = subscribedTopicDescriber;
+        this.subscriptionIds = new 
HashSet<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+        this.rackInfo = new RackInfo(assignmentSpec, subscribedTopicDescriber, 
subscriptionIds);
+        this.potentiallyUnfilledMembers = new HashMap<>();
+        this.unfilledMembers = new HashMap<>();
+        this.newAssignment = new HashMap<>();
+        this.useRackAwareStrategy = rackInfo.useRackStrategy;
+        // Without rack-aware strategy, tracking current owners of unassigned 
partitions is unnecessary
+        // as all sticky partitions are retained until a member meets its 
quota.
+        this.currentPartitionOwners = useRackAwareStrategy ? new HashMap<>() : 
Collections.emptyMap();
+    }
+
+    /**
+     * Here's the step-by-step breakdown of the assignment process:
+     *
+     * <li> Compute the quotas of partitions for each member based on the 
total partitions and member count.</li>
+     * <li> For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.</li>
+     * <li> If a partition's rack mismatches with its owner, track it for 
future use.</li>
+     * <li> Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.</li>
+     * <li> Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.</li>
+     * <li> Depending on members needing extra partitions, select members from 
the potentially unfilled list
+     *      and add them to the unfilled list.</li>
+     * <li> Proceed with a round-robin assignment adhering to rack awareness.
+     *      For each unassigned partition, locate the first compatible member 
from the unfilled list.</li>
+     * <li> If no rack-compatible member is found, revert to the tracked 
current owner.
+     *      If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.</li>
+     * */
+    @Override
+    protected GroupAssignment buildAssignment() throws 
PartitionAssignorException{
+        int totalPartitionsCount = 0;
+        // Removes the current topic from subscriptionIds if the topic doesn't 
exist in the topic metadata.
+        for (Uuid topicId : subscriptionIds) {
+            int partitionCount = 
subscribedTopicDescriber.numPartitions(topicId);
+            if (partitionCount == -1) {
+                IllegalStateException illegalStateException =
+                    new IllegalStateException("The topic " + topicId + " seems 
to be in an invalid state.");

Review Comment:
   I think that we should directly throw the IllegalStateException. It is a bit 
weird to construct it to wrap it in another one right away.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+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.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * <ul>
+ *     <li>
+ *         <b> Optimized Uniform Assignment Builder: </b> This strategy is 
used when all members have subscribed
+ *         to the same set of topics.
+ *     </li>
+ *     <li>
+ *         <b> General Uniform Assignment Builder: </b> This strategy is used 
when members have varied topic
+ *         subscriptions.
+ *     </li>
+ * </ul>
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+    private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+    public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+    @Override
+    public String name() {
+        return UNIFORM_ASSIGNOR_NAME;
+    }
+
+    /**
+     * Perform the group assignment given the current members and
+     * topic metadata.
+     *
+     * @param assignmentSpec                The member assignment spec.
+     * @param subscribedTopicDescriber      The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+     * @return The new assignment for the group.
+     */
+    @Override
+    public GroupAssignment assign(
+        AssignmentSpec assignmentSpec,
+        SubscribedTopicDescriber subscribedTopicDescriber
+    ) throws PartitionAssignorException {
+        AbstractAssignmentBuilder assignmentBuilder;
+        if (assignmentSpec.members().isEmpty())
+            return new GroupAssignment(Collections.emptyMap());
+
+        if (allSubscriptionsEqual(assignmentSpec.members())) {
+            log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
+                + "optimized assignment algorithm");
+            assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+        } else {
+            assignmentBuilder = new GeneralUniformAssignmentBuilder();
+            log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
+                + "general assignment algorithm");
+        }
+
+        return assignmentBuilder.buildAssignment();
+    }
+
+    /**
+     * Determines if all members are subscribed to the same list of topic IDs.
+     *
+     * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+     *                Assumes the map is non-empty.
+     * @return true if all members have the same subscription list of topic 
IDs,
+     *         false otherwise.
+     */
+    private boolean allSubscriptionsEqual(Map<String, AssignmentMemberSpec> 
members) {
+        Set<Uuid> firstSubscriptionSet = new 
HashSet<>(members.values().iterator().next().subscribedTopicIds());
+        for (AssignmentMemberSpec memberSpec : members.values()) {
+            Set<Uuid> currentSubscriptionSet = new 
HashSet<>(memberSpec.subscribedTopicIds());
+            if (!firstSubscriptionSet.equals(currentSubscriptionSet)) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    /**
+     * The assignment builder is used to construct the final assignment in a 
series of steps that
+     * are determined by the type of subscriptions.
+     *
+     * There are common methods present that are used by any type of 
assignment strategy.
+     */
+     protected static abstract class AbstractAssignmentBuilder {
+        protected abstract GroupAssignment buildAssignment();
+
+        /**
+         * Determines if rack-aware assignment is appropriate based on the 
provided rack information.
+         *
+         * @param memberRacks           Racks where members are located.
+         * @param partitionRacks        Racks where partitions are located.
+         * @param racksPerPartition     Map of partitions to their associated 
racks.
+         *
+         * @return {@code true} if rack-aware assignment should be applied; 
{@code false} otherwise.
+         */
+        protected boolean useRackAwareAssignment(
+            Set<String> memberRacks,
+            Set<String> partitionRacks,
+            Map<TopicIdPartition, Set<String>> racksPerPartition
+        ) {
+            if (memberRacks.isEmpty() || Collections.disjoint(memberRacks, 
partitionRacks))
+                return false;
+            else {
+                return 
!racksPerPartition.values().stream().allMatch(partitionRacks::equals);
+            }
+        }
+
+        /**
+         * Constructs a list of {@code TopicIdPartition} for each topic Id 
based on its partition count.
+         *
+         * @param allTopicIds                   The list of subscribed topic 
Ids.
+         * @param subscribedTopicDescriber      Utility to fetch the partition 
count for a given topic.
+         *
+         * @return List of generated {@code TopicIdPartition} for all provided 
topic Ids.
+         */
+        protected List<TopicIdPartition> allTopicIdPartitions(
+            List<Uuid> allTopicIds,
+            SubscribedTopicDescriber subscribedTopicDescriber
+        ) {
+            List<TopicIdPartition> allTopicIdPartitions = new ArrayList<>();
+            allTopicIds.forEach(topic ->
+                IntStream.range(0, 
subscribedTopicDescriber.numPartitions((topic)))
+                    .forEach(i -> allTopicIdPartitions.add(new 
TopicIdPartition(topic, i))
+                )
+            );
+            
+            return allTopicIdPartitions;
+        }
+
+        /**
+         * Represents the rack information of members and partitions along 
with utility methods
+         * to facilitate rack-aware assignment strategies for a given consumer 
group.
+         */
+        protected class RackInfo {
+            /**
+             * Map of every member to its rack.
+             */
+            protected final Map<String, String> memberRacks;
+            /**
+             * Map of every partition to a list of its racks.
+             */
+            protected final Map<TopicIdPartition, Set<String>> partitionRacks;
+            /**
+             * Number of members with the same rack as the partition.
+             */
+            private final Map<TopicIdPartition, Integer> 
numMembersWithSameRackByPartition;
+            protected final boolean useRackStrategy;

Review Comment:
   nit: javadoc.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *      balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+    private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+    private final AssignmentSpec assignmentSpec;
+    private final SubscribedTopicDescriber subscribedTopicDescriber;
+    // List of topics subscribed to by all members.
+    private final Set<Uuid> subscriptionIds;
+    private final RackInfo rackInfo;
+    // Count of members to receive an extra partition beyond the minimum quota,
+    // to account for the distribution of the remaining partitions.
+    private int remainingMembersToGetAnExtraPartition;
+    // Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+    // including members eligible for an extra partition.
+    private final Map<String, Integer> potentiallyUnfilledMembers;
+    // Members mapped to the remaining number of partitions needed to meet the 
full quota.
+    // Full quota = minQuota + one extra partition (if applicable).
+    private Map<String, Integer> unfilledMembers;
+    private List<TopicIdPartition> unassignedPartitions;
+    private final Map<String, MemberAssignment> newAssignment;
+    // Tracks the current owner of each partition when using rack-aware 
strategy.
+    // Current refers to the existing assignment.
+    private final Map<TopicIdPartition, String> currentPartitionOwners;
+    // Indicates if a rack aware assignment can be done.
+    // True if racks are defined for both members and partitions.
+    boolean useRackAwareStrategy;
+
+    OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+        this.assignmentSpec = assignmentSpec;
+        this.subscribedTopicDescriber = subscribedTopicDescriber;
+        this.subscriptionIds = new 
HashSet<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+        this.rackInfo = new RackInfo(assignmentSpec, subscribedTopicDescriber, 
subscriptionIds);
+        this.potentiallyUnfilledMembers = new HashMap<>();
+        this.unfilledMembers = new HashMap<>();
+        this.newAssignment = new HashMap<>();
+        this.useRackAwareStrategy = rackInfo.useRackStrategy;
+        // Without rack-aware strategy, tracking current owners of unassigned 
partitions is unnecessary
+        // as all sticky partitions are retained until a member meets its 
quota.
+        this.currentPartitionOwners = useRackAwareStrategy ? new HashMap<>() : 
Collections.emptyMap();
+    }
+
+    /**
+     * Here's the step-by-step breakdown of the assignment process:
+     *
+     * <li> Compute the quotas of partitions for each member based on the 
total partitions and member count.</li>
+     * <li> For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.</li>
+     * <li> If a partition's rack mismatches with its owner, track it for 
future use.</li>
+     * <li> Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.</li>
+     * <li> Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.</li>
+     * <li> Depending on members needing extra partitions, select members from 
the potentially unfilled list
+     *      and add them to the unfilled list.</li>
+     * <li> Proceed with a round-robin assignment adhering to rack awareness.
+     *      For each unassigned partition, locate the first compatible member 
from the unfilled list.</li>
+     * <li> If no rack-compatible member is found, revert to the tracked 
current owner.
+     *      If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.</li>
+     * */
+    @Override
+    protected GroupAssignment buildAssignment() throws 
PartitionAssignorException{
+        int totalPartitionsCount = 0;
+        // Removes the current topic from subscriptionIds if the topic doesn't 
exist in the topic metadata.
+        for (Uuid topicId : subscriptionIds) {
+            int partitionCount = 
subscribedTopicDescriber.numPartitions(topicId);
+            if (partitionCount == -1) {
+                IllegalStateException illegalStateException =
+                    new IllegalStateException("The topic " + topicId + " seems 
to be in an invalid state.");
+
+                throw new PartitionAssignorException(
+                    "Members are subscribed to topic " + topicId + " which 
doesn't exist in the topic metadata.",
+                    illegalStateException
+                );
+            } else {
+                totalPartitionsCount += partitionCount;
+            }
+        }
+
+        if (subscriptionIds.isEmpty()) {
+            log.info("The subscription list is empty, returning an empty 
assignment");
+            return new GroupAssignment(Collections.emptyMap());
+        }
+
+        // The minimum required quota that each member needs to meet for a 
balanced assignment.
+        // This is the same for all members.
+        int numberOfMembers = assignmentSpec.members().size();

Review Comment:
   Could this be final as well?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *      balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+    private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);

Review Comment:
   nit: `log` -> `LOG`.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+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.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * <ul>
+ *     <li>
+ *         <b> Optimized Uniform Assignment Builder: </b> This strategy is 
used when all members have subscribed
+ *         to the same set of topics.
+ *     </li>
+ *     <li>
+ *         <b> General Uniform Assignment Builder: </b> This strategy is used 
when members have varied topic
+ *         subscriptions.
+ *     </li>
+ * </ul>
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+    private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+    public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+    @Override
+    public String name() {
+        return UNIFORM_ASSIGNOR_NAME;
+    }
+
+    /**
+     * Perform the group assignment given the current members and
+     * topic metadata.
+     *
+     * @param assignmentSpec                The member assignment spec.
+     * @param subscribedTopicDescriber      The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+     * @return The new assignment for the group.
+     */
+    @Override
+    public GroupAssignment assign(
+        AssignmentSpec assignmentSpec,
+        SubscribedTopicDescriber subscribedTopicDescriber
+    ) throws PartitionAssignorException {
+        AbstractAssignmentBuilder assignmentBuilder;
+        if (assignmentSpec.members().isEmpty())
+            return new GroupAssignment(Collections.emptyMap());
+
+        if (allSubscriptionsEqual(assignmentSpec.members())) {
+            log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
+                + "optimized assignment algorithm");
+            assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+        } else {
+            assignmentBuilder = new GeneralUniformAssignmentBuilder();
+            log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
+                + "general assignment algorithm");
+        }
+
+        return assignmentBuilder.buildAssignment();
+    }
+
+    /**
+     * Determines if all members are subscribed to the same list of topic IDs.
+     *
+     * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+     *                Assumes the map is non-empty.
+     * @return true if all members have the same subscription list of topic 
IDs,
+     *         false otherwise.
+     */
+    private boolean allSubscriptionsEqual(Map<String, AssignmentMemberSpec> 
members) {
+        Set<Uuid> firstSubscriptionSet = new 
HashSet<>(members.values().iterator().next().subscribedTopicIds());
+        for (AssignmentMemberSpec memberSpec : members.values()) {
+            Set<Uuid> currentSubscriptionSet = new 
HashSet<>(memberSpec.subscribedTopicIds());
+            if (!firstSubscriptionSet.equals(currentSubscriptionSet)) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    /**
+     * The assignment builder is used to construct the final assignment in a 
series of steps that
+     * are determined by the type of subscriptions.
+     *
+     * There are common methods present that are used by any type of 
assignment strategy.
+     */
+     protected static abstract class AbstractAssignmentBuilder {
+        protected abstract GroupAssignment buildAssignment();
+
+        /**
+         * Determines if rack-aware assignment is appropriate based on the 
provided rack information.
+         *
+         * @param memberRacks           Racks where members are located.
+         * @param partitionRacks        Racks where partitions are located.
+         * @param racksPerPartition     Map of partitions to their associated 
racks.
+         *
+         * @return {@code true} if rack-aware assignment should be applied; 
{@code false} otherwise.
+         */
+        protected boolean useRackAwareAssignment(
+            Set<String> memberRacks,
+            Set<String> partitionRacks,
+            Map<TopicIdPartition, Set<String>> racksPerPartition
+        ) {
+            if (memberRacks.isEmpty() || Collections.disjoint(memberRacks, 
partitionRacks))
+                return false;
+            else {
+                return 
!racksPerPartition.values().stream().allMatch(partitionRacks::equals);
+            }
+        }
+
+        /**
+         * Constructs a list of {@code TopicIdPartition} for each topic Id 
based on its partition count.
+         *
+         * @param allTopicIds                   The list of subscribed topic 
Ids.
+         * @param subscribedTopicDescriber      Utility to fetch the partition 
count for a given topic.
+         *
+         * @return List of generated {@code TopicIdPartition} for all provided 
topic Ids.
+         */
+        protected List<TopicIdPartition> allTopicIdPartitions(
+            List<Uuid> allTopicIds,
+            SubscribedTopicDescriber subscribedTopicDescriber
+        ) {
+            List<TopicIdPartition> allTopicIdPartitions = new ArrayList<>();
+            allTopicIds.forEach(topic ->
+                IntStream.range(0, 
subscribedTopicDescriber.numPartitions((topic)))
+                    .forEach(i -> allTopicIdPartitions.add(new 
TopicIdPartition(topic, i))
+                )
+            );
+            
+            return allTopicIdPartitions;
+        }
+
+        /**
+         * Represents the rack information of members and partitions along 
with utility methods
+         * to facilitate rack-aware assignment strategies for a given consumer 
group.
+         */
+        protected class RackInfo {
+            /**
+             * Map of every member to its rack.
+             */
+            protected final Map<String, String> memberRacks;
+            /**
+             * Map of every partition to a list of its racks.
+             */
+            protected final Map<TopicIdPartition, Set<String>> partitionRacks;
+            /**
+             * Number of members with the same rack as the partition.
+             */
+            private final Map<TopicIdPartition, Integer> 
numMembersWithSameRackByPartition;
+            protected final boolean useRackStrategy;
+
+            /**
+             * Constructs rack information based on assignment specification 
and subscribed topics.
+             *
+             * @param assignmentSpec                The current assignment 
specification.
+             * @param subscribedTopicDescriber      Topic and partition 
metadata of the subscribed topics.
+             * @param topicIds                      List of topic Ids.
+             */
+            public RackInfo(
+                AssignmentSpec assignmentSpec,
+                SubscribedTopicDescriber subscribedTopicDescriber,
+                Set<Uuid> topicIds
+            ) {
+                Map<String, List<String>> membersByRack = new HashMap<>();
+                assignmentSpec.members().forEach((memberId, 
assignmentMemberSpec) ->
+                    assignmentMemberSpec.rackId().filter(r -> 
!r.isEmpty()).ifPresent(
+                        rackId -> membersByRack.computeIfAbsent(rackId, __ -> 
new ArrayList<>()).add(memberId)
+                    )
+                );
+
+                Set<String> allPartitionRacks;
+                Map<TopicIdPartition, Set<String>> partitionRacks;
+                List<TopicIdPartition> topicIdPartitions = 
allTopicIdPartitions(new ArrayList<>(topicIds), subscribedTopicDescriber);

Review Comment:
   nit: Do we really need to create the ArrayList here?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+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.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * <ul>
+ *     <li>
+ *         <b> Optimized Uniform Assignment Builder: </b> This strategy is 
used when all members have subscribed
+ *         to the same set of topics.
+ *     </li>
+ *     <li>
+ *         <b> General Uniform Assignment Builder: </b> This strategy is used 
when members have varied topic
+ *         subscriptions.
+ *     </li>
+ * </ul>
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+    private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);

Review Comment:
   nit: `log` -> `LOG`.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+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.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * <ul>
+ *     <li>
+ *         <b> Optimized Uniform Assignment Builder: </b> This strategy is 
used when all members have subscribed
+ *         to the same set of topics.
+ *     </li>
+ *     <li>
+ *         <b> General Uniform Assignment Builder: </b> This strategy is used 
when members have varied topic
+ *         subscriptions.
+ *     </li>
+ * </ul>
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+    private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+    public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+    @Override
+    public String name() {
+        return UNIFORM_ASSIGNOR_NAME;
+    }
+
+    /**
+     * Perform the group assignment given the current members and
+     * topic metadata.
+     *
+     * @param assignmentSpec                The member assignment spec.
+     * @param subscribedTopicDescriber      The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+     * @return The new assignment for the group.
+     */
+    @Override
+    public GroupAssignment assign(
+        AssignmentSpec assignmentSpec,
+        SubscribedTopicDescriber subscribedTopicDescriber
+    ) throws PartitionAssignorException {
+        AbstractAssignmentBuilder assignmentBuilder;
+        if (assignmentSpec.members().isEmpty())
+            return new GroupAssignment(Collections.emptyMap());
+
+        if (allSubscriptionsEqual(assignmentSpec.members())) {
+            log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
+                + "optimized assignment algorithm");
+            assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+        } else {
+            assignmentBuilder = new GeneralUniformAssignmentBuilder();
+            log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
+                + "general assignment algorithm");
+        }
+
+        return assignmentBuilder.buildAssignment();
+    }
+
+    /**
+     * Determines if all members are subscribed to the same list of topic IDs.
+     *
+     * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+     *                Assumes the map is non-empty.
+     * @return true if all members have the same subscription list of topic 
IDs,
+     *         false otherwise.
+     */
+    private boolean allSubscriptionsEqual(Map<String, AssignmentMemberSpec> 
members) {
+        Set<Uuid> firstSubscriptionSet = new 
HashSet<>(members.values().iterator().next().subscribedTopicIds());
+        for (AssignmentMemberSpec memberSpec : members.values()) {
+            Set<Uuid> currentSubscriptionSet = new 
HashSet<>(memberSpec.subscribedTopicIds());
+            if (!firstSubscriptionSet.equals(currentSubscriptionSet)) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    /**
+     * The assignment builder is used to construct the final assignment in a 
series of steps that
+     * are determined by the type of subscriptions.
+     *
+     * There are common methods present that are used by any type of 
assignment strategy.
+     */
+     protected static abstract class AbstractAssignmentBuilder {
+        protected abstract GroupAssignment buildAssignment();
+
+        /**
+         * Determines if rack-aware assignment is appropriate based on the 
provided rack information.
+         *
+         * @param memberRacks           Racks where members are located.
+         * @param partitionRacks        Racks where partitions are located.
+         * @param racksPerPartition     Map of partitions to their associated 
racks.
+         *
+         * @return {@code true} if rack-aware assignment should be applied; 
{@code false} otherwise.
+         */
+        protected boolean useRackAwareAssignment(

Review Comment:
   nit: Could this method be static?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *      balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+    private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+    private final AssignmentSpec assignmentSpec;
+    private final SubscribedTopicDescriber subscribedTopicDescriber;
+    // List of topics subscribed to by all members.
+    private final Set<Uuid> subscriptionIds;
+    private final RackInfo rackInfo;
+    // Count of members to receive an extra partition beyond the minimum quota,
+    // to account for the distribution of the remaining partitions.
+    private int remainingMembersToGetAnExtraPartition;
+    // Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+    // including members eligible for an extra partition.
+    private final Map<String, Integer> potentiallyUnfilledMembers;
+    // Members mapped to the remaining number of partitions needed to meet the 
full quota.
+    // Full quota = minQuota + one extra partition (if applicable).
+    private Map<String, Integer> unfilledMembers;
+    private List<TopicIdPartition> unassignedPartitions;
+    private final Map<String, MemberAssignment> newAssignment;
+    // Tracks the current owner of each partition when using rack-aware 
strategy.
+    // Current refers to the existing assignment.
+    private final Map<TopicIdPartition, String> currentPartitionOwners;
+    // Indicates if a rack aware assignment can be done.
+    // True if racks are defined for both members and partitions.
+    boolean useRackAwareStrategy;
+
+    OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+        this.assignmentSpec = assignmentSpec;
+        this.subscribedTopicDescriber = subscribedTopicDescriber;
+        this.subscriptionIds = new 
HashSet<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+        this.rackInfo = new RackInfo(assignmentSpec, subscribedTopicDescriber, 
subscriptionIds);
+        this.potentiallyUnfilledMembers = new HashMap<>();
+        this.unfilledMembers = new HashMap<>();
+        this.newAssignment = new HashMap<>();
+        this.useRackAwareStrategy = rackInfo.useRackStrategy;
+        // Without rack-aware strategy, tracking current owners of unassigned 
partitions is unnecessary
+        // as all sticky partitions are retained until a member meets its 
quota.
+        this.currentPartitionOwners = useRackAwareStrategy ? new HashMap<>() : 
Collections.emptyMap();
+    }
+
+    /**
+     * Here's the step-by-step breakdown of the assignment process:
+     *
+     * <li> Compute the quotas of partitions for each member based on the 
total partitions and member count.</li>
+     * <li> For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.</li>
+     * <li> If a partition's rack mismatches with its owner, track it for 
future use.</li>
+     * <li> Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.</li>
+     * <li> Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.</li>
+     * <li> Depending on members needing extra partitions, select members from 
the potentially unfilled list
+     *      and add them to the unfilled list.</li>
+     * <li> Proceed with a round-robin assignment adhering to rack awareness.
+     *      For each unassigned partition, locate the first compatible member 
from the unfilled list.</li>
+     * <li> If no rack-compatible member is found, revert to the tracked 
current owner.
+     *      If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.</li>
+     * */
+    @Override
+    protected GroupAssignment buildAssignment() throws 
PartitionAssignorException{
+        int totalPartitionsCount = 0;
+        // Removes the current topic from subscriptionIds if the topic doesn't 
exist in the topic metadata.

Review Comment:
   This comment is incorrect as the implementation fails if the topic does not 
exist.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *      balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+    private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+    private final AssignmentSpec assignmentSpec;
+    private final SubscribedTopicDescriber subscribedTopicDescriber;
+    // List of topics subscribed to by all members.
+    private final Set<Uuid> subscriptionIds;
+    private final RackInfo rackInfo;
+    // Count of members to receive an extra partition beyond the minimum quota,
+    // to account for the distribution of the remaining partitions.
+    private int remainingMembersToGetAnExtraPartition;
+    // Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+    // including members eligible for an extra partition.
+    private final Map<String, Integer> potentiallyUnfilledMembers;
+    // Members mapped to the remaining number of partitions needed to meet the 
full quota.
+    // Full quota = minQuota + one extra partition (if applicable).
+    private Map<String, Integer> unfilledMembers;
+    private List<TopicIdPartition> unassignedPartitions;
+    private final Map<String, MemberAssignment> newAssignment;
+    // Tracks the current owner of each partition when using rack-aware 
strategy.
+    // Current refers to the existing assignment.
+    private final Map<TopicIdPartition, String> currentPartitionOwners;
+    // Indicates if a rack aware assignment can be done.
+    // True if racks are defined for both members and partitions.
+    boolean useRackAwareStrategy;
+
+    OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+        this.assignmentSpec = assignmentSpec;
+        this.subscribedTopicDescriber = subscribedTopicDescriber;
+        this.subscriptionIds = new 
HashSet<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+        this.rackInfo = new RackInfo(assignmentSpec, subscribedTopicDescriber, 
subscriptionIds);
+        this.potentiallyUnfilledMembers = new HashMap<>();
+        this.unfilledMembers = new HashMap<>();
+        this.newAssignment = new HashMap<>();
+        this.useRackAwareStrategy = rackInfo.useRackStrategy;
+        // Without rack-aware strategy, tracking current owners of unassigned 
partitions is unnecessary
+        // as all sticky partitions are retained until a member meets its 
quota.
+        this.currentPartitionOwners = useRackAwareStrategy ? new HashMap<>() : 
Collections.emptyMap();
+    }
+
+    /**
+     * Here's the step-by-step breakdown of the assignment process:
+     *
+     * <li> Compute the quotas of partitions for each member based on the 
total partitions and member count.</li>
+     * <li> For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.</li>
+     * <li> If a partition's rack mismatches with its owner, track it for 
future use.</li>
+     * <li> Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.</li>
+     * <li> Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.</li>
+     * <li> Depending on members needing extra partitions, select members from 
the potentially unfilled list
+     *      and add them to the unfilled list.</li>
+     * <li> Proceed with a round-robin assignment adhering to rack awareness.
+     *      For each unassigned partition, locate the first compatible member 
from the unfilled list.</li>
+     * <li> If no rack-compatible member is found, revert to the tracked 
current owner.
+     *      If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.</li>
+     * */

Review Comment:
   nit: There is an space between the `*`.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/common/TopicIdPartition.java:
##########
@@ -0,0 +1,74 @@
+/*

Review Comment:
   We already have the very same class in the `metadata` module. I wonder if we 
should move that one to `server-common` module so that we could reuse it here. 
It could land in the `common` package over there. What do you think?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+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.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * <ul>
+ *     <li>
+ *         <b> Optimized Uniform Assignment Builder: </b> This strategy is 
used when all members have subscribed
+ *         to the same set of topics.
+ *     </li>
+ *     <li>
+ *         <b> General Uniform Assignment Builder: </b> This strategy is used 
when members have varied topic
+ *         subscriptions.
+ *     </li>
+ * </ul>
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+    private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+    public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+    @Override
+    public String name() {
+        return UNIFORM_ASSIGNOR_NAME;
+    }
+
+    /**
+     * Perform the group assignment given the current members and
+     * topic metadata.
+     *
+     * @param assignmentSpec                The member assignment spec.
+     * @param subscribedTopicDescriber      The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+     * @return The new assignment for the group.
+     */
+    @Override
+    public GroupAssignment assign(
+        AssignmentSpec assignmentSpec,
+        SubscribedTopicDescriber subscribedTopicDescriber
+    ) throws PartitionAssignorException {
+        AbstractAssignmentBuilder assignmentBuilder;
+        if (assignmentSpec.members().isEmpty())
+            return new GroupAssignment(Collections.emptyMap());
+
+        if (allSubscriptionsEqual(assignmentSpec.members())) {
+            log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
+                + "optimized assignment algorithm");
+            assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+        } else {
+            assignmentBuilder = new GeneralUniformAssignmentBuilder();
+            log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
+                + "general assignment algorithm");
+        }
+
+        return assignmentBuilder.buildAssignment();
+    }
+
+    /**
+     * Determines if all members are subscribed to the same list of topic IDs.
+     *
+     * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+     *                Assumes the map is non-empty.
+     * @return true if all members have the same subscription list of topic 
IDs,
+     *         false otherwise.
+     */
+    private boolean allSubscriptionsEqual(Map<String, AssignmentMemberSpec> 
members) {
+        Set<Uuid> firstSubscriptionSet = new 
HashSet<>(members.values().iterator().next().subscribedTopicIds());
+        for (AssignmentMemberSpec memberSpec : members.values()) {
+            Set<Uuid> currentSubscriptionSet = new 
HashSet<>(memberSpec.subscribedTopicIds());
+            if (!firstSubscriptionSet.equals(currentSubscriptionSet)) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    /**
+     * The assignment builder is used to construct the final assignment in a 
series of steps that
+     * are determined by the type of subscriptions.
+     *
+     * There are common methods present that are used by any type of 
assignment strategy.
+     */
+     protected static abstract class AbstractAssignmentBuilder {

Review Comment:
   Is there a reason to have this abstract class here? Personally, I find it a 
bit weird as the concrete classes are not defined in this class. How about 
extracting it?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *      balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+    private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+    private final AssignmentSpec assignmentSpec;
+    private final SubscribedTopicDescriber subscribedTopicDescriber;
+    // List of topics subscribed to by all members.
+    private final Set<Uuid> subscriptionIds;
+    private final RackInfo rackInfo;
+    // Count of members to receive an extra partition beyond the minimum quota,
+    // to account for the distribution of the remaining partitions.
+    private int remainingMembersToGetAnExtraPartition;
+    // Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+    // including members eligible for an extra partition.
+    private final Map<String, Integer> potentiallyUnfilledMembers;
+    // Members mapped to the remaining number of partitions needed to meet the 
full quota.
+    // Full quota = minQuota + one extra partition (if applicable).
+    private Map<String, Integer> unfilledMembers;
+    private List<TopicIdPartition> unassignedPartitions;
+    private final Map<String, MemberAssignment> newAssignment;
+    // Tracks the current owner of each partition when using rack-aware 
strategy.
+    // Current refers to the existing assignment.
+    private final Map<TopicIdPartition, String> currentPartitionOwners;
+    // Indicates if a rack aware assignment can be done.
+    // True if racks are defined for both members and partitions.
+    boolean useRackAwareStrategy;
+
+    OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+        this.assignmentSpec = assignmentSpec;
+        this.subscribedTopicDescriber = subscribedTopicDescriber;
+        this.subscriptionIds = new 
HashSet<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+        this.rackInfo = new RackInfo(assignmentSpec, subscribedTopicDescriber, 
subscriptionIds);
+        this.potentiallyUnfilledMembers = new HashMap<>();
+        this.unfilledMembers = new HashMap<>();
+        this.newAssignment = new HashMap<>();
+        this.useRackAwareStrategy = rackInfo.useRackStrategy;
+        // Without rack-aware strategy, tracking current owners of unassigned 
partitions is unnecessary
+        // as all sticky partitions are retained until a member meets its 
quota.
+        this.currentPartitionOwners = useRackAwareStrategy ? new HashMap<>() : 
Collections.emptyMap();
+    }
+
+    /**
+     * Here's the step-by-step breakdown of the assignment process:
+     *
+     * <li> Compute the quotas of partitions for each member based on the 
total partitions and member count.</li>
+     * <li> For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.</li>
+     * <li> If a partition's rack mismatches with its owner, track it for 
future use.</li>
+     * <li> Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.</li>
+     * <li> Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.</li>
+     * <li> Depending on members needing extra partitions, select members from 
the potentially unfilled list
+     *      and add them to the unfilled list.</li>
+     * <li> Proceed with a round-robin assignment adhering to rack awareness.
+     *      For each unassigned partition, locate the first compatible member 
from the unfilled list.</li>
+     * <li> If no rack-compatible member is found, revert to the tracked 
current owner.
+     *      If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.</li>
+     * */
+    @Override
+    protected GroupAssignment buildAssignment() throws 
PartitionAssignorException{
+        int totalPartitionsCount = 0;
+        // Removes the current topic from subscriptionIds if the topic doesn't 
exist in the topic metadata.
+        for (Uuid topicId : subscriptionIds) {
+            int partitionCount = 
subscribedTopicDescriber.numPartitions(topicId);
+            if (partitionCount == -1) {
+                IllegalStateException illegalStateException =
+                    new IllegalStateException("The topic " + topicId + " seems 
to be in an invalid state.");
+
+                throw new PartitionAssignorException(
+                    "Members are subscribed to topic " + topicId + " which 
doesn't exist in the topic metadata.",
+                    illegalStateException
+                );
+            } else {
+                totalPartitionsCount += partitionCount;
+            }
+        }
+
+        if (subscriptionIds.isEmpty()) {
+            log.info("The subscription list is empty, returning an empty 
assignment");
+            return new GroupAssignment(Collections.emptyMap());
+        }
+
+        // The minimum required quota that each member needs to meet for a 
balanced assignment.
+        // This is the same for all members.
+        int numberOfMembers = assignmentSpec.members().size();
+        final int minQuota = totalPartitionsCount / numberOfMembers;
+        remainingMembersToGetAnExtraPartition = totalPartitionsCount % 
numberOfMembers;
+
+        assignmentSpec.members().keySet().forEach(memberId ->
+            newAssignment.put(memberId, new MemberAssignment(new HashMap<>())
+        ));
+
+        Set<TopicIdPartition> allAssignedStickyPartitions = 
computeAssignedStickyPartitions(minQuota);
+        unassignedPartitions = 
computeUnassignedPartitions(allAssignedStickyPartitions);
+        unfilledMembers = computeUnfilledMembers();
+
+        if (!unassignedPartitionsCountEqualsRemainingAssignmentsCount()) {
+            throw new PartitionAssignorException("Number of available 
partitions is not equal to the total requirement");
+        }
+
+        if (useRackAwareStrategy) rackAwareRoundRobinAssignment();
+        unassignedPartitionsRoundRobinAssignment();
+
+        return new GroupAssignment(newAssignment);
+    }
+
+    /**
+     * Retrieves a set of partitions that were currently assigned to members 
and will be retained in the new assignment,
+     * by ensuring that the partitions are still relevant based on current 
topic metadata and subscriptions.
+     * If rack awareness is enabled, it ensures that a partition's rack 
matches the member's rack.
+     *
+     * <p> For each member, it:
+     * <ul>
+     *     <li> Finds the valid current assignment considering topic 
subscriptions and metadata.</li>
+     *     <li> If current assignments exist, retains up to the minimum quota 
of assignments.</li>
+     *     <li> If there are members that should get an extra partition,
+     *          assigns the next partition after the retained ones.</li>
+     *     <li> For members with assignments not exceeding the minimum quota,
+     *          it identifies them as potentially unfilled members and tracks 
the remaining quota.</li>
+     * </ul>
+     *
+     * @return A set containing all the sticky partitions that have been 
retained in the new assignment.
+     */
+    private Set<TopicIdPartition> computeAssignedStickyPartitions(Integer 
minQuota) {

Review Comment:
   nit: We should use `int` here.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##########
@@ -0,0 +1,393 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *      balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+    private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+    private final AssignmentSpec assignmentSpec;
+    private final SubscribedTopicDescriber subscribedTopicDescriber;
+    // List of topics subscribed to by all members.
+    private final Set<Uuid> subscriptionIds;
+    private final RackInfo rackInfo;
+    // Count of members to receive an extra partition beyond the minimum quota,
+    // to account for the distribution of the remaining partitions.
+    private int remainingMembersToGetAnExtraPartition;
+    // Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+    // including members eligible for an extra partition.
+    private final Map<String, Integer> potentiallyUnfilledMembers;
+    // Members mapped to the remaining number of partitions needed to meet the 
full quota.
+    // Full quota = minQuota + one extra partition (if applicable).
+    private Map<String, Integer> unfilledMembers;
+    private List<TopicIdPartition> unassignedPartitions;
+    private final Map<String, MemberAssignment> newAssignment;
+    // Tracks the current owner of each partition when using rack-aware 
strategy.
+    // Current refers to the existing assignment.
+    private final Map<TopicIdPartition, String> currentPartitionOwners;
+    // Indicates if a rack aware assignment can be done.
+    // True if racks are defined for both members and partitions.
+    boolean useRackAwareStrategy;
+
+    OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+        this.assignmentSpec = assignmentSpec;
+        this.subscribedTopicDescriber = subscribedTopicDescriber;
+        this.subscriptionIds = new 
HashSet<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+        this.rackInfo = new RackInfo(assignmentSpec, subscribedTopicDescriber, 
subscriptionIds);
+        this.potentiallyUnfilledMembers = new HashMap<>();
+        this.unfilledMembers = new HashMap<>();
+        this.newAssignment = new HashMap<>();
+        this.useRackAwareStrategy = rackInfo.useRackStrategy;
+        // Without rack-aware strategy, tracking current owners of unassigned 
partitions is unnecessary
+        // as all sticky partitions are retained until a member meets its 
quota.
+        this.currentPartitionOwners = useRackAwareStrategy ? new HashMap<>() : 
Collections.emptyMap();
+    }
+
+    /**
+     * Here's the step-by-step breakdown of the assignment process:
+     *
+     * <li> Compute the quotas of partitions for each member based on the 
total partitions and member count.</li>
+     * <li> For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.</li>
+     * <li> If a partition's rack mismatches with its owner, track it for 
future use.</li>
+     * <li> Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.</li>
+     * <li> Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.</li>
+     * <li> Depending on members needing extra partitions, select members from 
the potentially unfilled list
+     *      and add them to the unfilled list.</li>
+     * <li> Proceed with a round-robin assignment adhering to rack awareness.
+     *      For each unassigned partition, locate the first compatible member 
from the unfilled list.</li>
+     * <li> If no rack-compatible member is found, revert to the tracked 
current owner.
+     *      If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.</li>
+     * */
+    @Override
+    protected GroupAssignment buildAssignment() throws 
PartitionAssignorException{
+        int totalPartitionsCount = 0;
+        // Removes the current topic from subscriptionIds if the topic doesn't 
exist in the topic metadata.
+        for (Uuid topicId : subscriptionIds) {
+            int partitionCount = 
subscribedTopicDescriber.numPartitions(topicId);
+            if (partitionCount == -1) {
+                IllegalStateException illegalStateException =
+                    new IllegalStateException("The topic " + topicId + " seems 
to be in an invalid state.");
+
+                throw new PartitionAssignorException(
+                    "Members are subscribed to topic " + topicId + " which 
doesn't exist in the topic metadata.",
+                    illegalStateException
+                );
+            } else {
+                totalPartitionsCount += partitionCount;
+            }
+        }
+
+        if (subscriptionIds.isEmpty()) {
+            log.info("The subscription list is empty, returning an empty 
assignment");
+            return new GroupAssignment(Collections.emptyMap());
+        }
+
+        // The minimum required quota that each member needs to meet for a 
balanced assignment.
+        // This is the same for all members.
+        int numberOfMembers = assignmentSpec.members().size();
+        final int minQuota = totalPartitionsCount / numberOfMembers;
+        remainingMembersToGetAnExtraPartition = totalPartitionsCount % 
numberOfMembers;
+
+        assignmentSpec.members().keySet().forEach(memberId ->
+            newAssignment.put(memberId, new MemberAssignment(new HashMap<>())
+        ));
+
+        Set<TopicIdPartition> allAssignedStickyPartitions = 
computeAssignedStickyPartitions(minQuota);
+        unassignedPartitions = 
computeUnassignedPartitions(allAssignedStickyPartitions);
+        unfilledMembers = computeUnfilledMembers();
+
+        if (!unassignedPartitionsCountEqualsRemainingAssignmentsCount()) {
+            throw new PartitionAssignorException("Number of available 
partitions is not equal to the total requirement");
+        }
+
+        if (useRackAwareStrategy) rackAwareRoundRobinAssignment();
+        unassignedPartitionsRoundRobinAssignment();
+
+        return new GroupAssignment(newAssignment);
+    }
+
+    /**
+     * Retrieves a set of partitions that were currently assigned to members 
and will be retained in the new assignment,
+     * by ensuring that the partitions are still relevant based on current 
topic metadata and subscriptions.
+     * If rack awareness is enabled, it ensures that a partition's rack 
matches the member's rack.
+     *
+     * <p> For each member, it:
+     * <ul>
+     *     <li> Finds the valid current assignment considering topic 
subscriptions and metadata.</li>
+     *     <li> If current assignments exist, retains up to the minimum quota 
of assignments.</li>
+     *     <li> If there are members that should get an extra partition,
+     *          assigns the next partition after the retained ones.</li>
+     *     <li> For members with assignments not exceeding the minimum quota,
+     *          it identifies them as potentially unfilled members and tracks 
the remaining quota.</li>
+     * </ul>
+     *
+     * @return A set containing all the sticky partitions that have been 
retained in the new assignment.
+     */
+    private Set<TopicIdPartition> computeAssignedStickyPartitions(Integer 
minQuota) {
+        Set<TopicIdPartition> allAssignedStickyPartitions = new HashSet<>();
+
+        assignmentSpec.members().forEach((memberId, assignmentMemberSpec) -> {
+            // Remove all the topics that aren't in the subscriptions or the 
topic metadata anymore.
+            // If rack awareness is enabled, only add partitions if the 
members rack matches the partitions rack.
+            List<TopicIdPartition> validCurrentAssignment = 
validCurrentAssignment(
+                memberId,
+                assignmentMemberSpec.assignedPartitions()
+            );
+
+            int currentAssignmentSize = validCurrentAssignment.size();
+            int remaining = minQuota - currentAssignmentSize;
+
+            if (currentAssignmentSize > 0) {
+                int retainedPartitionsCount = min(currentAssignmentSize, 
minQuota);
+                IntStream.range(0, retainedPartitionsCount).forEach(i -> {
+                    newAssignment.get(memberId)
+                        .targetPartitions()
+                        
.computeIfAbsent(validCurrentAssignment.get(i).topicId(), __ -> new HashSet<>())
+                        .add(validCurrentAssignment.get(i).partition());
+                    
allAssignedStickyPartitions.add(validCurrentAssignment.get(i));
+                });
+
+                // The extra partition is located at the last index from the 
previous step.
+                if (remaining < 0 && remainingMembersToGetAnExtraPartition > 
0) {
+                    newAssignment.get(memberId)
+                        .targetPartitions()
+                        
.computeIfAbsent(validCurrentAssignment.get(retainedPartitionsCount).topicId(), 
__ -> new HashSet<>())
+                        
.add(validCurrentAssignment.get(retainedPartitionsCount).partition());

Review Comment:
   nit: This code is duplicated. Should we have an helper method for it?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java:
##########
@@ -49,6 +51,8 @@
  * </ol>
  */
 public class RangeAssignor implements PartitionAssignor {
+    private static final Logger log = 
LoggerFactory.getLogger(RangeAssignor.class);

Review Comment:
   nit: `log` -> `LOG`.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##########
@@ -0,0 +1,288 @@
+/*
+ * 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.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+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.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * <ul>
+ *     <li>
+ *         <b> Optimized Uniform Assignment Builder: </b> This strategy is 
used when all members have subscribed
+ *         to the same set of topics.
+ *     </li>
+ *     <li>
+ *         <b> General Uniform Assignment Builder: </b> This strategy is used 
when members have varied topic
+ *         subscriptions.
+ *     </li>
+ * </ul>
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+    private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+    public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+    @Override
+    public String name() {
+        return UNIFORM_ASSIGNOR_NAME;
+    }
+
+    /**
+     * Perform the group assignment given the current members and
+     * topic metadata.
+     *
+     * @param assignmentSpec                The member assignment spec.
+     * @param subscribedTopicDescriber      The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+     * @return The new assignment for the group.
+     */
+    @Override
+    public GroupAssignment assign(
+        AssignmentSpec assignmentSpec,
+        SubscribedTopicDescriber subscribedTopicDescriber
+    ) throws PartitionAssignorException {
+        AbstractAssignmentBuilder assignmentBuilder;
+        if (assignmentSpec.members().isEmpty())
+            return new GroupAssignment(Collections.emptyMap());
+
+        if (allSubscriptionsEqual(assignmentSpec.members())) {
+            log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
+                + "optimized assignment algorithm");
+            assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+        } else {
+            assignmentBuilder = new GeneralUniformAssignmentBuilder();
+            log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
+                + "general assignment algorithm");
+        }
+
+        return assignmentBuilder.buildAssignment();
+    }
+
+    /**
+     * Determines if all members are subscribed to the same list of topic IDs.
+     *
+     * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+     *                Assumes the map is non-empty.
+     * @return true if all members have the same subscription list of topic 
IDs,
+     *         false otherwise.
+     */
+    private boolean allSubscriptionsEqual(Map<String, AssignmentMemberSpec> 
members) {
+        Set<Uuid> firstSubscriptionSet = new 
HashSet<>(members.values().iterator().next().subscribedTopicIds());
+        for (AssignmentMemberSpec memberSpec : members.values()) {
+            Set<Uuid> currentSubscriptionSet = new 
HashSet<>(memberSpec.subscribedTopicIds());
+            if (!firstSubscriptionSet.equals(currentSubscriptionSet)) {
+                return false;
+            }
+        }
+        return true;
+    }
+
+    /**
+     * The assignment builder is used to construct the final assignment in a 
series of steps that
+     * are determined by the type of subscriptions.
+     *
+     * There are common methods present that are used by any type of 
assignment strategy.
+     */
+     protected static abstract class AbstractAssignmentBuilder {
+        protected abstract GroupAssignment buildAssignment();
+
+        /**
+         * Determines if rack-aware assignment is appropriate based on the 
provided rack information.
+         *
+         * @param memberRacks           Racks where members are located.
+         * @param partitionRacks        Racks where partitions are located.
+         * @param racksPerPartition     Map of partitions to their associated 
racks.
+         *
+         * @return {@code true} if rack-aware assignment should be applied; 
{@code false} otherwise.
+         */
+        protected boolean useRackAwareAssignment(
+            Set<String> memberRacks,
+            Set<String> partitionRacks,
+            Map<TopicIdPartition, Set<String>> racksPerPartition
+        ) {
+            if (memberRacks.isEmpty() || Collections.disjoint(memberRacks, 
partitionRacks))
+                return false;
+            else {
+                return 
!racksPerPartition.values().stream().allMatch(partitionRacks::equals);
+            }
+        }
+
+        /**
+         * Constructs a list of {@code TopicIdPartition} for each topic Id 
based on its partition count.
+         *
+         * @param allTopicIds                   The list of subscribed topic 
Ids.
+         * @param subscribedTopicDescriber      Utility to fetch the partition 
count for a given topic.
+         *
+         * @return List of generated {@code TopicIdPartition} for all provided 
topic Ids.
+         */
+        protected List<TopicIdPartition> allTopicIdPartitions(

Review Comment:
   ditto.



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