apoorvmittal10 commented on code in PR #18864:
URL: https://github.com/apache/kafka/pull/18864#discussion_r1965651474


##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java:
##########
@@ -67,42 +71,210 @@ private GroupAssignment assignHomogenous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Set<Uuid> subscribeTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
+        Set<Uuid> subscribedTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
             .subscribedTopicIds();
-        if (subscribeTopicIds.isEmpty())
-            return new GroupAssignment(Collections.emptyMap());
+        if (subscribedTopicIds.isEmpty())
+            return new GroupAssignment(Map.of());
 
-        Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
-            subscribeTopicIds, subscribedTopicDescriber);
+        // Subscribed topic partitions for the share group.
+        List<TopicIdPartition> targetPartitions = computeTargetPartitions(
+            subscribedTopicIds, subscribedTopicDescriber);
 
-        return new 
GroupAssignment(groupSpec.memberIds().stream().collect(Collectors.toMap(
-            Function.identity(), memberId -> new 
MemberAssignmentImpl(targetPartitions))));
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHomogeneous(groupSpec, subscribedTopicIds, 
targetPartitions, currentAssignment);
     }
 
     private GroupAssignment assignHeterogeneous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Map<String, MemberAssignment> members = new HashMap<>();
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription = 
new HashMap<>();
         for (String memberId : groupSpec.memberIds()) {
             MemberSubscription spec = groupSpec.memberSubscription(memberId);
             if (spec.subscribedTopicIds().isEmpty())
                 continue;
 
-            Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
+            // Subscribed topic partitions for the share group member.
+            List<TopicIdPartition> targetPartitions = computeTargetPartitions(
                 spec.subscribedTopicIds(), subscribedTopicDescriber);
+            memberToPartitionsSubscription.put(memberId, targetPartitions);
+        }
+
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHeterogeneous(groupSpec, 
memberToPartitionsSubscription, currentAssignment);
+    }
+
+    /**
+     * Get the current assignment by topic partitions.
+     * @param groupSpec - The group metadata specifications.
+     * @return the current assignment for subscribed topic partitions to 
memberIds.
+     */
+    private Map<TopicIdPartition, List<String>> currentAssignment(GroupSpec 
groupSpec) {
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
 
-            members.put(memberId, new MemberAssignmentImpl(targetPartitions));
+        for (String member : groupSpec.memberIds()) {
+            Map<Uuid, Set<Integer>> assignedTopicPartitions = 
groupSpec.memberAssignment(member).partitions();
+            assignedTopicPartitions.forEach((topicId, partitions) -> 
partitions.forEach(
+                partition -> assignment.computeIfAbsent(new 
TopicIdPartition(topicId, partition), k -> new ArrayList<>()).add(member)));
         }
+        return assignment;
+    }
+
+    private GroupAssignment newAssignmentHomogeneous(
+        GroupSpec groupSpec,
+        Set<Uuid> subscribedTopicIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+

Review Comment:
   nit: remove line break please



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java:
##########
@@ -67,42 +71,210 @@ private GroupAssignment assignHomogenous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Set<Uuid> subscribeTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
+        Set<Uuid> subscribedTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
             .subscribedTopicIds();
-        if (subscribeTopicIds.isEmpty())
-            return new GroupAssignment(Collections.emptyMap());
+        if (subscribedTopicIds.isEmpty())
+            return new GroupAssignment(Map.of());
 
-        Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
-            subscribeTopicIds, subscribedTopicDescriber);
+        // Subscribed topic partitions for the share group.
+        List<TopicIdPartition> targetPartitions = computeTargetPartitions(
+            subscribedTopicIds, subscribedTopicDescriber);
 
-        return new 
GroupAssignment(groupSpec.memberIds().stream().collect(Collectors.toMap(
-            Function.identity(), memberId -> new 
MemberAssignmentImpl(targetPartitions))));
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHomogeneous(groupSpec, subscribedTopicIds, 
targetPartitions, currentAssignment);
     }
 
     private GroupAssignment assignHeterogeneous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Map<String, MemberAssignment> members = new HashMap<>();
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription = 
new HashMap<>();
         for (String memberId : groupSpec.memberIds()) {
             MemberSubscription spec = groupSpec.memberSubscription(memberId);
             if (spec.subscribedTopicIds().isEmpty())
                 continue;
 
-            Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
+            // Subscribed topic partitions for the share group member.
+            List<TopicIdPartition> targetPartitions = computeTargetPartitions(
                 spec.subscribedTopicIds(), subscribedTopicDescriber);
+            memberToPartitionsSubscription.put(memberId, targetPartitions);
+        }
+
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHeterogeneous(groupSpec, 
memberToPartitionsSubscription, currentAssignment);
+    }
+
+    /**
+     * Get the current assignment by topic partitions.
+     * @param groupSpec - The group metadata specifications.
+     * @return the current assignment for subscribed topic partitions to 
memberIds.
+     */
+    private Map<TopicIdPartition, List<String>> currentAssignment(GroupSpec 
groupSpec) {
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
 
-            members.put(memberId, new MemberAssignmentImpl(targetPartitions));
+        for (String member : groupSpec.memberIds()) {
+            Map<Uuid, Set<Integer>> assignedTopicPartitions = 
groupSpec.memberAssignment(member).partitions();
+            assignedTopicPartitions.forEach((topicId, partitions) -> 
partitions.forEach(
+                partition -> assignment.computeIfAbsent(new 
TopicIdPartition(topicId, partition), k -> new ArrayList<>()).add(member)));
         }
+        return assignment;
+    }
+
+    private GroupAssignment newAssignmentHomogeneous(
+        GroupSpec groupSpec,
+        Set<Uuid> subscribedTopicIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberHashAssignment(targetPartitions, groupSpec.memberIds(), 
newAssignment);
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = newAssignment.keySet();
+        List<TopicIdPartition> unassignedPartitions = targetPartitions.stream()
+            .filter(targetPartition -> 
!assignedPartitions.contains(targetPartition))
+            .filter(targetPartition -> 
!currentAssignment.containsKey(targetPartition))
+            .toList();
+
+        roundRobinAssignment(groupSpec.memberIds(), unassignedPartitions, 
newAssignment);
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
topics in current assignment that are also being
+        // subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> {
+            if (subscribedTopicIds.contains(targetPartition.topicId()))
+                members.forEach(member -> {
+                    if (groupSpec.memberIds().contains(member))
+                        finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+                });
+        });
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment newAssignmentHeterogeneous(
+        GroupSpec groupSpec,
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        // Exhaustive set of all subscribed topic partitions.
+        Set<TopicIdPartition> targetPartitions = new LinkedHashSet<>();
+        
memberToPartitionsSubscription.values().forEach(targetPartitions::addAll);
+
+        // Create a map for topic to members subscription.
+        Map<Uuid, Set<String>> topicToMemberSubscription = new HashMap<>();
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            partitions.forEach(partition -> 
topicToMemberSubscription.computeIfAbsent(partition.topicId(), k -> new 
LinkedHashSet<>()).add(member)));
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            memberHashAssignment(partitions, List.of(member), newAssignment));
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = new 
LinkedHashSet<>(newAssignment.keySet());
+        Map<Uuid, List<TopicIdPartition>> unassignedPartitions = new 
HashMap<>();
+        targetPartitions.forEach(targetPartition -> {
+            if (!assignedPartitions.contains(targetPartition) && 
!currentAssignment.containsKey(targetPartition))
+                
unassignedPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
ArrayList<>()).add(targetPartition);
+        });
+
+        unassignedPartitions.keySet().forEach(unassignedTopic ->
+            
roundRobinAssignment(topicToMemberSubscription.get(unassignedTopic), 
unassignedPartitions.get(unassignedTopic), newAssignment));
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
member topic subscription in current assignment
+        // which is being subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> 
members.forEach(member -> {
+            if 
(topicToMemberSubscription.getOrDefault(targetPartition.topicId(), 
Collections.emptySet()).contains(member))
+                finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+        }));
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment groupAssignment(
+        Map<String, Set<TopicIdPartition>> assignmentByMember,
+        Collection<String> allGroupMembers
+    ) {
+        Map<String, MemberAssignment> members = new HashMap<>();
+        for (Map.Entry<String, Set<TopicIdPartition>> entry : 
assignmentByMember.entrySet()) {
+            Map<Uuid, Set<Integer>> targetPartitions = new HashMap<>();
+            entry.getValue().forEach(targetPartition -> 
targetPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
HashSet<>()).add(targetPartition.partitionId()));
+            members.put(entry.getKey(), new 
MemberAssignmentImpl(targetPartitions));
+        }
+        allGroupMembers.forEach(member -> {
+            if (!members.containsKey(member))
+                members.put(member, new MemberAssignmentImpl(new HashMap<>()));
+        });
+
         return new GroupAssignment(members);
     }
 
-    private Map<Uuid, Set<Integer>> computeTargetPartitions(
-        Set<Uuid> subscribeTopicIds,
+    /**
+     * This function updates assignment by hashing the member IDs of the 
members and maps the partitions assigned to the members based on the hash. This 
gives approximately even balance.

Review Comment:
   Please break it in 2 lines.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java:
##########
@@ -67,42 +71,210 @@ private GroupAssignment assignHomogenous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Set<Uuid> subscribeTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
+        Set<Uuid> subscribedTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
             .subscribedTopicIds();
-        if (subscribeTopicIds.isEmpty())
-            return new GroupAssignment(Collections.emptyMap());
+        if (subscribedTopicIds.isEmpty())
+            return new GroupAssignment(Map.of());
 
-        Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
-            subscribeTopicIds, subscribedTopicDescriber);
+        // Subscribed topic partitions for the share group.
+        List<TopicIdPartition> targetPartitions = computeTargetPartitions(
+            subscribedTopicIds, subscribedTopicDescriber);
 
-        return new 
GroupAssignment(groupSpec.memberIds().stream().collect(Collectors.toMap(
-            Function.identity(), memberId -> new 
MemberAssignmentImpl(targetPartitions))));
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHomogeneous(groupSpec, subscribedTopicIds, 
targetPartitions, currentAssignment);
     }
 
     private GroupAssignment assignHeterogeneous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Map<String, MemberAssignment> members = new HashMap<>();
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription = 
new HashMap<>();
         for (String memberId : groupSpec.memberIds()) {
             MemberSubscription spec = groupSpec.memberSubscription(memberId);
             if (spec.subscribedTopicIds().isEmpty())
                 continue;
 
-            Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
+            // Subscribed topic partitions for the share group member.
+            List<TopicIdPartition> targetPartitions = computeTargetPartitions(
                 spec.subscribedTopicIds(), subscribedTopicDescriber);
+            memberToPartitionsSubscription.put(memberId, targetPartitions);
+        }
+
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHeterogeneous(groupSpec, 
memberToPartitionsSubscription, currentAssignment);
+    }
+
+    /**
+     * Get the current assignment by topic partitions.
+     * @param groupSpec - The group metadata specifications.
+     * @return the current assignment for subscribed topic partitions to 
memberIds.
+     */
+    private Map<TopicIdPartition, List<String>> currentAssignment(GroupSpec 
groupSpec) {
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
 
-            members.put(memberId, new MemberAssignmentImpl(targetPartitions));
+        for (String member : groupSpec.memberIds()) {
+            Map<Uuid, Set<Integer>> assignedTopicPartitions = 
groupSpec.memberAssignment(member).partitions();
+            assignedTopicPartitions.forEach((topicId, partitions) -> 
partitions.forEach(
+                partition -> assignment.computeIfAbsent(new 
TopicIdPartition(topicId, partition), k -> new ArrayList<>()).add(member)));
         }
+        return assignment;
+    }
+
+    private GroupAssignment newAssignmentHomogeneous(
+        GroupSpec groupSpec,
+        Set<Uuid> subscribedTopicIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberHashAssignment(targetPartitions, groupSpec.memberIds(), 
newAssignment);
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = newAssignment.keySet();
+        List<TopicIdPartition> unassignedPartitions = targetPartitions.stream()
+            .filter(targetPartition -> 
!assignedPartitions.contains(targetPartition))
+            .filter(targetPartition -> 
!currentAssignment.containsKey(targetPartition))
+            .toList();
+
+        roundRobinAssignment(groupSpec.memberIds(), unassignedPartitions, 
newAssignment);
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
topics in current assignment that are also being
+        // subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> {
+            if (subscribedTopicIds.contains(targetPartition.topicId()))
+                members.forEach(member -> {
+                    if (groupSpec.memberIds().contains(member))
+                        finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+                });
+        });
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment newAssignmentHeterogeneous(
+        GroupSpec groupSpec,
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        // Exhaustive set of all subscribed topic partitions.
+        Set<TopicIdPartition> targetPartitions = new LinkedHashSet<>();
+        
memberToPartitionsSubscription.values().forEach(targetPartitions::addAll);
+
+        // Create a map for topic to members subscription.
+        Map<Uuid, Set<String>> topicToMemberSubscription = new HashMap<>();
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            partitions.forEach(partition -> 
topicToMemberSubscription.computeIfAbsent(partition.topicId(), k -> new 
LinkedHashSet<>()).add(member)));
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            memberHashAssignment(partitions, List.of(member), newAssignment));
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = new 
LinkedHashSet<>(newAssignment.keySet());
+        Map<Uuid, List<TopicIdPartition>> unassignedPartitions = new 
HashMap<>();
+        targetPartitions.forEach(targetPartition -> {
+            if (!assignedPartitions.contains(targetPartition) && 
!currentAssignment.containsKey(targetPartition))
+                
unassignedPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
ArrayList<>()).add(targetPartition);
+        });
+
+        unassignedPartitions.keySet().forEach(unassignedTopic ->
+            
roundRobinAssignment(topicToMemberSubscription.get(unassignedTopic), 
unassignedPartitions.get(unassignedTopic), newAssignment));
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
member topic subscription in current assignment
+        // which is being subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> 
members.forEach(member -> {
+            if 
(topicToMemberSubscription.getOrDefault(targetPartition.topicId(), 
Collections.emptySet()).contains(member))
+                finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+        }));
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment groupAssignment(
+        Map<String, Set<TopicIdPartition>> assignmentByMember,
+        Collection<String> allGroupMembers
+    ) {
+        Map<String, MemberAssignment> members = new HashMap<>();
+        for (Map.Entry<String, Set<TopicIdPartition>> entry : 
assignmentByMember.entrySet()) {
+            Map<Uuid, Set<Integer>> targetPartitions = new HashMap<>();
+            entry.getValue().forEach(targetPartition -> 
targetPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
HashSet<>()).add(targetPartition.partitionId()));
+            members.put(entry.getKey(), new 
MemberAssignmentImpl(targetPartitions));
+        }
+        allGroupMembers.forEach(member -> {
+            if (!members.containsKey(member))
+                members.put(member, new MemberAssignmentImpl(new HashMap<>()));
+        });
+
         return new GroupAssignment(members);
     }
 
-    private Map<Uuid, Set<Integer>> computeTargetPartitions(
-        Set<Uuid> subscribeTopicIds,
+    /**
+     * This function updates assignment by hashing the member IDs of the 
members and maps the partitions assigned to the members based on the hash. This 
gives approximately even balance.
+     * @param targetPartitions - the subscribed topic partitions.
+     * @param memberIds - the member ids to which the topic partitions need to 
be assigned.
+     * @param assignment - the existing assignment by topic partition. We need 
to pass it as a parameter because this
+     *                   function would be called multiple times for 
heterogeneous assignment.
+     */
+    void memberHashAssignment(
+        List<TopicIdPartition> targetPartitions,

Review Comment:
   ```suggestion
           List<TopicIdPartition> unassignedPartitions,
   ```



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java:
##########
@@ -67,42 +71,210 @@ private GroupAssignment assignHomogenous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Set<Uuid> subscribeTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
+        Set<Uuid> subscribedTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
             .subscribedTopicIds();
-        if (subscribeTopicIds.isEmpty())
-            return new GroupAssignment(Collections.emptyMap());
+        if (subscribedTopicIds.isEmpty())
+            return new GroupAssignment(Map.of());
 
-        Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
-            subscribeTopicIds, subscribedTopicDescriber);
+        // Subscribed topic partitions for the share group.
+        List<TopicIdPartition> targetPartitions = computeTargetPartitions(
+            subscribedTopicIds, subscribedTopicDescriber);
 
-        return new 
GroupAssignment(groupSpec.memberIds().stream().collect(Collectors.toMap(
-            Function.identity(), memberId -> new 
MemberAssignmentImpl(targetPartitions))));
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHomogeneous(groupSpec, subscribedTopicIds, 
targetPartitions, currentAssignment);
     }
 
     private GroupAssignment assignHeterogeneous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Map<String, MemberAssignment> members = new HashMap<>();
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription = 
new HashMap<>();
         for (String memberId : groupSpec.memberIds()) {
             MemberSubscription spec = groupSpec.memberSubscription(memberId);
             if (spec.subscribedTopicIds().isEmpty())
                 continue;
 
-            Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
+            // Subscribed topic partitions for the share group member.
+            List<TopicIdPartition> targetPartitions = computeTargetPartitions(
                 spec.subscribedTopicIds(), subscribedTopicDescriber);
+            memberToPartitionsSubscription.put(memberId, targetPartitions);
+        }
+
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHeterogeneous(groupSpec, 
memberToPartitionsSubscription, currentAssignment);
+    }
+
+    /**
+     * Get the current assignment by topic partitions.
+     * @param groupSpec - The group metadata specifications.
+     * @return the current assignment for subscribed topic partitions to 
memberIds.
+     */
+    private Map<TopicIdPartition, List<String>> currentAssignment(GroupSpec 
groupSpec) {
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
 
-            members.put(memberId, new MemberAssignmentImpl(targetPartitions));
+        for (String member : groupSpec.memberIds()) {
+            Map<Uuid, Set<Integer>> assignedTopicPartitions = 
groupSpec.memberAssignment(member).partitions();
+            assignedTopicPartitions.forEach((topicId, partitions) -> 
partitions.forEach(
+                partition -> assignment.computeIfAbsent(new 
TopicIdPartition(topicId, partition), k -> new ArrayList<>()).add(member)));
         }
+        return assignment;
+    }
+
+    private GroupAssignment newAssignmentHomogeneous(
+        GroupSpec groupSpec,
+        Set<Uuid> subscribedTopicIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberHashAssignment(targetPartitions, groupSpec.memberIds(), 
newAssignment);
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = newAssignment.keySet();
+        List<TopicIdPartition> unassignedPartitions = targetPartitions.stream()
+            .filter(targetPartition -> 
!assignedPartitions.contains(targetPartition))
+            .filter(targetPartition -> 
!currentAssignment.containsKey(targetPartition))
+            .toList();
+
+        roundRobinAssignment(groupSpec.memberIds(), unassignedPartitions, 
newAssignment);
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
topics in current assignment that are also being
+        // subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> {
+            if (subscribedTopicIds.contains(targetPartition.topicId()))
+                members.forEach(member -> {
+                    if (groupSpec.memberIds().contains(member))
+                        finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+                });
+        });
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment newAssignmentHeterogeneous(
+        GroupSpec groupSpec,
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        // Exhaustive set of all subscribed topic partitions.
+        Set<TopicIdPartition> targetPartitions = new LinkedHashSet<>();
+        
memberToPartitionsSubscription.values().forEach(targetPartitions::addAll);
+
+        // Create a map for topic to members subscription.
+        Map<Uuid, Set<String>> topicToMemberSubscription = new HashMap<>();
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            partitions.forEach(partition -> 
topicToMemberSubscription.computeIfAbsent(partition.topicId(), k -> new 
LinkedHashSet<>()).add(member)));
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            memberHashAssignment(partitions, List.of(member), newAssignment));
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = new 
LinkedHashSet<>(newAssignment.keySet());
+        Map<Uuid, List<TopicIdPartition>> unassignedPartitions = new 
HashMap<>();
+        targetPartitions.forEach(targetPartition -> {
+            if (!assignedPartitions.contains(targetPartition) && 
!currentAssignment.containsKey(targetPartition))
+                
unassignedPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
ArrayList<>()).add(targetPartition);
+        });
+
+        unassignedPartitions.keySet().forEach(unassignedTopic ->
+            
roundRobinAssignment(topicToMemberSubscription.get(unassignedTopic), 
unassignedPartitions.get(unassignedTopic), newAssignment));
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
member topic subscription in current assignment
+        // which is being subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> 
members.forEach(member -> {
+            if 
(topicToMemberSubscription.getOrDefault(targetPartition.topicId(), 
Collections.emptySet()).contains(member))
+                finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+        }));
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment groupAssignment(
+        Map<String, Set<TopicIdPartition>> assignmentByMember,
+        Collection<String> allGroupMembers
+    ) {
+        Map<String, MemberAssignment> members = new HashMap<>();
+        for (Map.Entry<String, Set<TopicIdPartition>> entry : 
assignmentByMember.entrySet()) {
+            Map<Uuid, Set<Integer>> targetPartitions = new HashMap<>();
+            entry.getValue().forEach(targetPartition -> 
targetPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
HashSet<>()).add(targetPartition.partitionId()));
+            members.put(entry.getKey(), new 
MemberAssignmentImpl(targetPartitions));
+        }
+        allGroupMembers.forEach(member -> {
+            if (!members.containsKey(member))
+                members.put(member, new MemberAssignmentImpl(new HashMap<>()));
+        });
+
         return new GroupAssignment(members);
     }
 
-    private Map<Uuid, Set<Integer>> computeTargetPartitions(
-        Set<Uuid> subscribeTopicIds,
+    /**
+     * This function updates assignment by hashing the member IDs of the 
members and maps the partitions assigned to the members based on the hash. This 
gives approximately even balance.
+     * @param targetPartitions - the subscribed topic partitions.
+     * @param memberIds - the member ids to which the topic partitions need to 
be assigned.
+     * @param assignment - the existing assignment by topic partition. We need 
to pass it as a parameter because this
+     *                   function would be called multiple times for 
heterogeneous assignment.
+     */
+    void memberHashAssignment(
+        List<TopicIdPartition> targetPartitions,
+        Collection<String> memberIds,
+        Map<TopicIdPartition, List<String>> assignment
+    ) {
+        if (!targetPartitions.isEmpty())
+            for (String memberId : memberIds) {
+                int topicPartitionIndex = Math.abs(memberId.hashCode() % 
targetPartitions.size());
+                TopicIdPartition topicPartition = 
targetPartitions.get(topicPartitionIndex);
+                assignment.computeIfAbsent(topicPartition, k -> new 
ArrayList<>()).add(memberId);
+            }
+    }
+
+    /**
+     * This functions assigns topic partitions to members by round-robin 
approach and updates the existing assignment.
+     * @param memberIds - the member ids to which the topic partitions need to 
be assigned.

Review Comment:
   ```suggestion
        * @param memberIds - the member ids to which the topic partitions need 
to be assigned, should be non-empty
   ```



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignorTest.java:
##########
@@ -320,12 +365,377 @@ public void 
testAssignWithOneMemberNoAssignedTopicHeterogeneous() {
         Map<String, Map<Uuid, Set<Integer>>> expectedAssignment = new 
HashMap<>();
         expectedAssignment.put(MEMBER_A, mkAssignment(
             mkTopicAssignment(TOPIC_1_UUID, 0, 1, 2),
-            mkTopicAssignment(TOPIC_2_UUID, 0, 1)
-        ));
+            mkTopicAssignment(TOPIC_2_UUID, 0, 1)));
+        expectedAssignment.put(MEMBER_B, mkAssignment());
 
+        // T1: 3 partitions + T2: 2 partitions = 5 partitions
+        assertEveryPartitionGetsAssignment(5, computedAssignment);
         assertAssignment(expectedAssignment, computedAssignment);
     }
 
+    @Test
+    public void testMemberHashAssignment() {
+        // hashcode for "member1" is 948881623.
+        String member1 = "member1";
+        // hashcode for "member2" is 948881624.
+        String member2 = "member2";
+        // hashcode for "member3" is 948881625.
+        String member3 = "member3";
+        // hashcode for "member4" is 948881626.
+        String member4 = "member4";
+        // hashcode for "AaAaAaAa" is -540425984 to test with negative 
hashcode.
+        String member5 = "AaAaAaAa";
+        List<String> members = Arrays.asList(member1, member2, member3, 
member4, member5);
+
+        TopicIdPartition partition1 = new TopicIdPartition(TOPIC_1_UUID, 0);
+        TopicIdPartition partition2 = new TopicIdPartition(TOPIC_2_UUID, 0);
+        TopicIdPartition partition3 = new TopicIdPartition(TOPIC_3_UUID, 0);
+        List<TopicIdPartition> partitions = Arrays.asList(partition1, 
partition2, partition3);

Review Comment:
   ```suggestion
           List<TopicIdPartition> partitions = List.of(partition1, partition2, 
partition3);
   ```



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignorTest.java:
##########
@@ -320,12 +365,377 @@ public void 
testAssignWithOneMemberNoAssignedTopicHeterogeneous() {
         Map<String, Map<Uuid, Set<Integer>>> expectedAssignment = new 
HashMap<>();
         expectedAssignment.put(MEMBER_A, mkAssignment(
             mkTopicAssignment(TOPIC_1_UUID, 0, 1, 2),
-            mkTopicAssignment(TOPIC_2_UUID, 0, 1)
-        ));
+            mkTopicAssignment(TOPIC_2_UUID, 0, 1)));
+        expectedAssignment.put(MEMBER_B, mkAssignment());
 
+        // T1: 3 partitions + T2: 2 partitions = 5 partitions
+        assertEveryPartitionGetsAssignment(5, computedAssignment);
         assertAssignment(expectedAssignment, computedAssignment);
     }
 
+    @Test
+    public void testMemberHashAssignment() {
+        // hashcode for "member1" is 948881623.
+        String member1 = "member1";
+        // hashcode for "member2" is 948881624.
+        String member2 = "member2";
+        // hashcode for "member3" is 948881625.
+        String member3 = "member3";
+        // hashcode for "member4" is 948881626.
+        String member4 = "member4";
+        // hashcode for "AaAaAaAa" is -540425984 to test with negative 
hashcode.
+        String member5 = "AaAaAaAa";
+        List<String> members = Arrays.asList(member1, member2, member3, 
member4, member5);
+
+        TopicIdPartition partition1 = new TopicIdPartition(TOPIC_1_UUID, 0);
+        TopicIdPartition partition2 = new TopicIdPartition(TOPIC_2_UUID, 0);
+        TopicIdPartition partition3 = new TopicIdPartition(TOPIC_3_UUID, 0);
+        List<TopicIdPartition> partitions = Arrays.asList(partition1, 
partition2, partition3);
+
+        Map<TopicIdPartition, List<String>> computedAssignment = new 
HashMap<>();
+        assignor.memberHashAssignment(partitions, members, computedAssignment);
+
+        Map<TopicIdPartition, List<String>> expectedAssignment = new 
HashMap<>();
+        expectedAssignment.put(partition1, List.of(member3));
+        expectedAssignment.put(partition2, Arrays.asList(member1, member4));
+        expectedAssignment.put(partition3, Arrays.asList(member2, member5));
+        assertAssignment(expectedAssignment, computedAssignment);
+    }
+
+    @Test
+    public void testRoundRobinAssignment() {
+        String member1 = "member1";
+        String member2 = "member2";
+        List<String> members = Arrays.asList(member1, member2);
+        TopicIdPartition partition1 = new TopicIdPartition(TOPIC_1_UUID, 0);
+        TopicIdPartition partition2 = new TopicIdPartition(TOPIC_2_UUID, 0);
+        TopicIdPartition partition3 = new TopicIdPartition(TOPIC_3_UUID, 0);
+        TopicIdPartition partition4 = new TopicIdPartition(TOPIC_4_UUID, 0);
+        List<TopicIdPartition> unassignedPartitions = 
Arrays.asList(partition2, partition3, partition4);
+
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
+        assignment.put(partition1, List.of(member1));
+
+        assignor.roundRobinAssignment(members, unassignedPartitions, 
assignment);
+        Map<TopicIdPartition, List<String>> expectedAssignment = new 
HashMap<>();
+        expectedAssignment.put(partition1, List.of(member1));
+        expectedAssignment.put(partition2, List.of(member1));
+        expectedAssignment.put(partition3, List.of(member2));
+        expectedAssignment.put(partition4, List.of(member1));
+
+        assertAssignment(expectedAssignment, assignment);
+    }
+
+    @Test
+    public void testAssignWithCurrentAssignmentHomogeneous() {
+        // Current assignment setup - Two members A, B subscribing to T1 and 
T2.
+        Map<Uuid, TopicMetadata> topicMetadata1 = new HashMap<>();
+        topicMetadata1.put(TOPIC_1_UUID, new TopicMetadata(
+            TOPIC_1_UUID,
+            TOPIC_1_NAME,
+            3
+        ));
+        topicMetadata1.put(TOPIC_2_UUID, new TopicMetadata(
+            TOPIC_2_UUID,
+            TOPIC_2_NAME,
+            2
+        ));
+
+        Map<String, MemberSubscriptionAndAssignmentImpl> members1 = new 
TreeMap<>();

Review Comment:
   Question: Why treemap and cosidering order? The hash and assignment should 
be predictable in tests?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java:
##########
@@ -67,42 +71,210 @@ private GroupAssignment assignHomogenous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Set<Uuid> subscribeTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
+        Set<Uuid> subscribedTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
             .subscribedTopicIds();
-        if (subscribeTopicIds.isEmpty())
-            return new GroupAssignment(Collections.emptyMap());
+        if (subscribedTopicIds.isEmpty())
+            return new GroupAssignment(Map.of());
 
-        Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
-            subscribeTopicIds, subscribedTopicDescriber);
+        // Subscribed topic partitions for the share group.
+        List<TopicIdPartition> targetPartitions = computeTargetPartitions(
+            subscribedTopicIds, subscribedTopicDescriber);
 
-        return new 
GroupAssignment(groupSpec.memberIds().stream().collect(Collectors.toMap(
-            Function.identity(), memberId -> new 
MemberAssignmentImpl(targetPartitions))));
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHomogeneous(groupSpec, subscribedTopicIds, 
targetPartitions, currentAssignment);
     }
 
     private GroupAssignment assignHeterogeneous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Map<String, MemberAssignment> members = new HashMap<>();
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription = 
new HashMap<>();
         for (String memberId : groupSpec.memberIds()) {
             MemberSubscription spec = groupSpec.memberSubscription(memberId);
             if (spec.subscribedTopicIds().isEmpty())
                 continue;
 
-            Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
+            // Subscribed topic partitions for the share group member.
+            List<TopicIdPartition> targetPartitions = computeTargetPartitions(
                 spec.subscribedTopicIds(), subscribedTopicDescriber);
+            memberToPartitionsSubscription.put(memberId, targetPartitions);
+        }
+
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHeterogeneous(groupSpec, 
memberToPartitionsSubscription, currentAssignment);
+    }
+
+    /**
+     * Get the current assignment by topic partitions.
+     * @param groupSpec - The group metadata specifications.
+     * @return the current assignment for subscribed topic partitions to 
memberIds.
+     */
+    private Map<TopicIdPartition, List<String>> currentAssignment(GroupSpec 
groupSpec) {
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
 
-            members.put(memberId, new MemberAssignmentImpl(targetPartitions));
+        for (String member : groupSpec.memberIds()) {
+            Map<Uuid, Set<Integer>> assignedTopicPartitions = 
groupSpec.memberAssignment(member).partitions();
+            assignedTopicPartitions.forEach((topicId, partitions) -> 
partitions.forEach(
+                partition -> assignment.computeIfAbsent(new 
TopicIdPartition(topicId, partition), k -> new ArrayList<>()).add(member)));
         }
+        return assignment;
+    }
+
+    private GroupAssignment newAssignmentHomogeneous(
+        GroupSpec groupSpec,
+        Set<Uuid> subscribedTopicIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+

Review Comment:
   nit: remove line break.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java:
##########
@@ -67,42 +71,210 @@ private GroupAssignment assignHomogenous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Set<Uuid> subscribeTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
+        Set<Uuid> subscribedTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
             .subscribedTopicIds();
-        if (subscribeTopicIds.isEmpty())
-            return new GroupAssignment(Collections.emptyMap());
+        if (subscribedTopicIds.isEmpty())
+            return new GroupAssignment(Map.of());
 
-        Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
-            subscribeTopicIds, subscribedTopicDescriber);
+        // Subscribed topic partitions for the share group.
+        List<TopicIdPartition> targetPartitions = computeTargetPartitions(
+            subscribedTopicIds, subscribedTopicDescriber);
 
-        return new 
GroupAssignment(groupSpec.memberIds().stream().collect(Collectors.toMap(
-            Function.identity(), memberId -> new 
MemberAssignmentImpl(targetPartitions))));
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHomogeneous(groupSpec, subscribedTopicIds, 
targetPartitions, currentAssignment);
     }
 
     private GroupAssignment assignHeterogeneous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Map<String, MemberAssignment> members = new HashMap<>();
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription = 
new HashMap<>();
         for (String memberId : groupSpec.memberIds()) {
             MemberSubscription spec = groupSpec.memberSubscription(memberId);
             if (spec.subscribedTopicIds().isEmpty())
                 continue;
 
-            Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
+            // Subscribed topic partitions for the share group member.
+            List<TopicIdPartition> targetPartitions = computeTargetPartitions(
                 spec.subscribedTopicIds(), subscribedTopicDescriber);
+            memberToPartitionsSubscription.put(memberId, targetPartitions);
+        }
+
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHeterogeneous(groupSpec, 
memberToPartitionsSubscription, currentAssignment);
+    }
+
+    /**
+     * Get the current assignment by topic partitions.
+     * @param groupSpec - The group metadata specifications.
+     * @return the current assignment for subscribed topic partitions to 
memberIds.
+     */
+    private Map<TopicIdPartition, List<String>> currentAssignment(GroupSpec 
groupSpec) {
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
 
-            members.put(memberId, new MemberAssignmentImpl(targetPartitions));
+        for (String member : groupSpec.memberIds()) {
+            Map<Uuid, Set<Integer>> assignedTopicPartitions = 
groupSpec.memberAssignment(member).partitions();
+            assignedTopicPartitions.forEach((topicId, partitions) -> 
partitions.forEach(
+                partition -> assignment.computeIfAbsent(new 
TopicIdPartition(topicId, partition), k -> new ArrayList<>()).add(member)));
         }
+        return assignment;
+    }
+
+    private GroupAssignment newAssignmentHomogeneous(
+        GroupSpec groupSpec,
+        Set<Uuid> subscribedTopicIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberHashAssignment(targetPartitions, groupSpec.memberIds(), 
newAssignment);
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = newAssignment.keySet();
+        List<TopicIdPartition> unassignedPartitions = targetPartitions.stream()
+            .filter(targetPartition -> 
!assignedPartitions.contains(targetPartition))
+            .filter(targetPartition -> 
!currentAssignment.containsKey(targetPartition))
+            .toList();
+
+        roundRobinAssignment(groupSpec.memberIds(), unassignedPartitions, 
newAssignment);
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
topics in current assignment that are also being
+        // subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> {
+            if (subscribedTopicIds.contains(targetPartition.topicId()))
+                members.forEach(member -> {
+                    if (groupSpec.memberIds().contains(member))
+                        finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+                });
+        });
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment newAssignmentHeterogeneous(
+        GroupSpec groupSpec,
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        // Exhaustive set of all subscribed topic partitions.
+        Set<TopicIdPartition> targetPartitions = new LinkedHashSet<>();
+        
memberToPartitionsSubscription.values().forEach(targetPartitions::addAll);
+
+        // Create a map for topic to members subscription.
+        Map<Uuid, Set<String>> topicToMemberSubscription = new HashMap<>();
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            partitions.forEach(partition -> 
topicToMemberSubscription.computeIfAbsent(partition.topicId(), k -> new 
LinkedHashSet<>()).add(member)));
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            memberHashAssignment(partitions, List.of(member), newAssignment));
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = new 
LinkedHashSet<>(newAssignment.keySet());
+        Map<Uuid, List<TopicIdPartition>> unassignedPartitions = new 
HashMap<>();
+        targetPartitions.forEach(targetPartition -> {
+            if (!assignedPartitions.contains(targetPartition) && 
!currentAssignment.containsKey(targetPartition))
+                
unassignedPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
ArrayList<>()).add(targetPartition);
+        });
+
+        unassignedPartitions.keySet().forEach(unassignedTopic ->
+            
roundRobinAssignment(topicToMemberSubscription.get(unassignedTopic), 
unassignedPartitions.get(unassignedTopic), newAssignment));
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
member topic subscription in current assignment
+        // which is being subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> 
members.forEach(member -> {
+            if 
(topicToMemberSubscription.getOrDefault(targetPartition.topicId(), 
Collections.emptySet()).contains(member))
+                finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+        }));
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment groupAssignment(
+        Map<String, Set<TopicIdPartition>> assignmentByMember,
+        Collection<String> allGroupMembers
+    ) {
+        Map<String, MemberAssignment> members = new HashMap<>();
+        for (Map.Entry<String, Set<TopicIdPartition>> entry : 
assignmentByMember.entrySet()) {
+            Map<Uuid, Set<Integer>> targetPartitions = new HashMap<>();
+            entry.getValue().forEach(targetPartition -> 
targetPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
HashSet<>()).add(targetPartition.partitionId()));
+            members.put(entry.getKey(), new 
MemberAssignmentImpl(targetPartitions));
+        }
+        allGroupMembers.forEach(member -> {
+            if (!members.containsKey(member))
+                members.put(member, new MemberAssignmentImpl(new HashMap<>()));
+        });
+
         return new GroupAssignment(members);
     }
 
-    private Map<Uuid, Set<Integer>> computeTargetPartitions(
-        Set<Uuid> subscribeTopicIds,
+    /**
+     * This function updates assignment by hashing the member IDs of the 
members and maps the partitions assigned to the members based on the hash. This 
gives approximately even balance.
+     * @param targetPartitions - the subscribed topic partitions.
+     * @param memberIds - the member ids to which the topic partitions need to 
be assigned.
+     * @param assignment - the existing assignment by topic partition. We need 
to pass it as a parameter because this
+     *                   function would be called multiple times for 
heterogeneous assignment.
+     */
+    void memberHashAssignment(
+        List<TopicIdPartition> targetPartitions,
+        Collection<String> memberIds,
+        Map<TopicIdPartition, List<String>> assignment
+    ) {
+        if (!targetPartitions.isEmpty())
+            for (String memberId : memberIds) {
+                int topicPartitionIndex = Math.abs(memberId.hashCode() % 
targetPartitions.size());

Review Comment:
   Query: We will not factor in any stickiness in assignments? I guess the 
future PRs will have a better version of stickiness, correct? Else there could 
be pre-fetched records in client and additional work to release records when 
assignment changes.
   cc: @AndrewJSchofield 



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java:
##########
@@ -67,42 +71,210 @@ private GroupAssignment assignHomogenous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Set<Uuid> subscribeTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
+        Set<Uuid> subscribedTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
             .subscribedTopicIds();
-        if (subscribeTopicIds.isEmpty())
-            return new GroupAssignment(Collections.emptyMap());
+        if (subscribedTopicIds.isEmpty())
+            return new GroupAssignment(Map.of());
 
-        Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
-            subscribeTopicIds, subscribedTopicDescriber);
+        // Subscribed topic partitions for the share group.
+        List<TopicIdPartition> targetPartitions = computeTargetPartitions(
+            subscribedTopicIds, subscribedTopicDescriber);
 
-        return new 
GroupAssignment(groupSpec.memberIds().stream().collect(Collectors.toMap(
-            Function.identity(), memberId -> new 
MemberAssignmentImpl(targetPartitions))));
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHomogeneous(groupSpec, subscribedTopicIds, 
targetPartitions, currentAssignment);
     }
 
     private GroupAssignment assignHeterogeneous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Map<String, MemberAssignment> members = new HashMap<>();
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription = 
new HashMap<>();
         for (String memberId : groupSpec.memberIds()) {
             MemberSubscription spec = groupSpec.memberSubscription(memberId);
             if (spec.subscribedTopicIds().isEmpty())
                 continue;
 
-            Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
+            // Subscribed topic partitions for the share group member.
+            List<TopicIdPartition> targetPartitions = computeTargetPartitions(
                 spec.subscribedTopicIds(), subscribedTopicDescriber);
+            memberToPartitionsSubscription.put(memberId, targetPartitions);
+        }
+
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHeterogeneous(groupSpec, 
memberToPartitionsSubscription, currentAssignment);
+    }
+
+    /**
+     * Get the current assignment by topic partitions.
+     * @param groupSpec - The group metadata specifications.
+     * @return the current assignment for subscribed topic partitions to 
memberIds.
+     */
+    private Map<TopicIdPartition, List<String>> currentAssignment(GroupSpec 
groupSpec) {
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
 
-            members.put(memberId, new MemberAssignmentImpl(targetPartitions));
+        for (String member : groupSpec.memberIds()) {
+            Map<Uuid, Set<Integer>> assignedTopicPartitions = 
groupSpec.memberAssignment(member).partitions();
+            assignedTopicPartitions.forEach((topicId, partitions) -> 
partitions.forEach(
+                partition -> assignment.computeIfAbsent(new 
TopicIdPartition(topicId, partition), k -> new ArrayList<>()).add(member)));
         }
+        return assignment;
+    }
+
+    private GroupAssignment newAssignmentHomogeneous(
+        GroupSpec groupSpec,
+        Set<Uuid> subscribedTopicIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.

Review Comment:
   ```suggestion
           // Step 1: Hash member IDs to topic partitions.
   ```



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java:
##########
@@ -67,42 +71,210 @@ private GroupAssignment assignHomogenous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Set<Uuid> subscribeTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
+        Set<Uuid> subscribedTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
             .subscribedTopicIds();
-        if (subscribeTopicIds.isEmpty())
-            return new GroupAssignment(Collections.emptyMap());
+        if (subscribedTopicIds.isEmpty())
+            return new GroupAssignment(Map.of());
 
-        Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
-            subscribeTopicIds, subscribedTopicDescriber);
+        // Subscribed topic partitions for the share group.
+        List<TopicIdPartition> targetPartitions = computeTargetPartitions(
+            subscribedTopicIds, subscribedTopicDescriber);
 
-        return new 
GroupAssignment(groupSpec.memberIds().stream().collect(Collectors.toMap(
-            Function.identity(), memberId -> new 
MemberAssignmentImpl(targetPartitions))));
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHomogeneous(groupSpec, subscribedTopicIds, 
targetPartitions, currentAssignment);
     }
 
     private GroupAssignment assignHeterogeneous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Map<String, MemberAssignment> members = new HashMap<>();
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription = 
new HashMap<>();
         for (String memberId : groupSpec.memberIds()) {
             MemberSubscription spec = groupSpec.memberSubscription(memberId);
             if (spec.subscribedTopicIds().isEmpty())
                 continue;
 
-            Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
+            // Subscribed topic partitions for the share group member.
+            List<TopicIdPartition> targetPartitions = computeTargetPartitions(
                 spec.subscribedTopicIds(), subscribedTopicDescriber);
+            memberToPartitionsSubscription.put(memberId, targetPartitions);
+        }
+
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHeterogeneous(groupSpec, 
memberToPartitionsSubscription, currentAssignment);
+    }
+
+    /**
+     * Get the current assignment by topic partitions.
+     * @param groupSpec - The group metadata specifications.
+     * @return the current assignment for subscribed topic partitions to 
memberIds.
+     */
+    private Map<TopicIdPartition, List<String>> currentAssignment(GroupSpec 
groupSpec) {
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
 
-            members.put(memberId, new MemberAssignmentImpl(targetPartitions));
+        for (String member : groupSpec.memberIds()) {
+            Map<Uuid, Set<Integer>> assignedTopicPartitions = 
groupSpec.memberAssignment(member).partitions();
+            assignedTopicPartitions.forEach((topicId, partitions) -> 
partitions.forEach(
+                partition -> assignment.computeIfAbsent(new 
TopicIdPartition(topicId, partition), k -> new ArrayList<>()).add(member)));
         }
+        return assignment;
+    }
+
+    private GroupAssignment newAssignmentHomogeneous(
+        GroupSpec groupSpec,
+        Set<Uuid> subscribedTopicIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberHashAssignment(targetPartitions, groupSpec.memberIds(), 
newAssignment);
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = newAssignment.keySet();

Review Comment:
   Why do we need separate variable to hold this? Can't we just call 
`newAssignment.containsKey` below?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java:
##########
@@ -67,42 +71,210 @@ private GroupAssignment assignHomogenous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Set<Uuid> subscribeTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
+        Set<Uuid> subscribedTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
             .subscribedTopicIds();
-        if (subscribeTopicIds.isEmpty())
-            return new GroupAssignment(Collections.emptyMap());
+        if (subscribedTopicIds.isEmpty())
+            return new GroupAssignment(Map.of());
 
-        Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
-            subscribeTopicIds, subscribedTopicDescriber);
+        // Subscribed topic partitions for the share group.
+        List<TopicIdPartition> targetPartitions = computeTargetPartitions(
+            subscribedTopicIds, subscribedTopicDescriber);
 
-        return new 
GroupAssignment(groupSpec.memberIds().stream().collect(Collectors.toMap(
-            Function.identity(), memberId -> new 
MemberAssignmentImpl(targetPartitions))));
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHomogeneous(groupSpec, subscribedTopicIds, 
targetPartitions, currentAssignment);
     }
 
     private GroupAssignment assignHeterogeneous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Map<String, MemberAssignment> members = new HashMap<>();
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription = 
new HashMap<>();
         for (String memberId : groupSpec.memberIds()) {
             MemberSubscription spec = groupSpec.memberSubscription(memberId);
             if (spec.subscribedTopicIds().isEmpty())
                 continue;
 
-            Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
+            // Subscribed topic partitions for the share group member.
+            List<TopicIdPartition> targetPartitions = computeTargetPartitions(
                 spec.subscribedTopicIds(), subscribedTopicDescriber);
+            memberToPartitionsSubscription.put(memberId, targetPartitions);
+        }
+
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHeterogeneous(groupSpec, 
memberToPartitionsSubscription, currentAssignment);
+    }
+
+    /**
+     * Get the current assignment by topic partitions.
+     * @param groupSpec - The group metadata specifications.
+     * @return the current assignment for subscribed topic partitions to 
memberIds.
+     */
+    private Map<TopicIdPartition, List<String>> currentAssignment(GroupSpec 
groupSpec) {
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
 
-            members.put(memberId, new MemberAssignmentImpl(targetPartitions));
+        for (String member : groupSpec.memberIds()) {
+            Map<Uuid, Set<Integer>> assignedTopicPartitions = 
groupSpec.memberAssignment(member).partitions();
+            assignedTopicPartitions.forEach((topicId, partitions) -> 
partitions.forEach(
+                partition -> assignment.computeIfAbsent(new 
TopicIdPartition(topicId, partition), k -> new ArrayList<>()).add(member)));
         }
+        return assignment;
+    }
+
+    private GroupAssignment newAssignmentHomogeneous(
+        GroupSpec groupSpec,
+        Set<Uuid> subscribedTopicIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberHashAssignment(targetPartitions, groupSpec.memberIds(), 
newAssignment);
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = newAssignment.keySet();
+        List<TopicIdPartition> unassignedPartitions = targetPartitions.stream()
+            .filter(targetPartition -> 
!assignedPartitions.contains(targetPartition))
+            .filter(targetPartition -> 
!currentAssignment.containsKey(targetPartition))
+            .toList();
+
+        roundRobinAssignment(groupSpec.memberIds(), unassignedPartitions, 
newAssignment);
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
topics in current assignment that are also being
+        // subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> {
+            if (subscribedTopicIds.contains(targetPartition.topicId()))
+                members.forEach(member -> {
+                    if (groupSpec.memberIds().contains(member))
+                        finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+                });
+        });
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment newAssignmentHeterogeneous(
+        GroupSpec groupSpec,
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        // Exhaustive set of all subscribed topic partitions.
+        Set<TopicIdPartition> targetPartitions = new LinkedHashSet<>();
+        
memberToPartitionsSubscription.values().forEach(targetPartitions::addAll);
+
+        // Create a map for topic to members subscription.
+        Map<Uuid, Set<String>> topicToMemberSubscription = new HashMap<>();
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            partitions.forEach(partition -> 
topicToMemberSubscription.computeIfAbsent(partition.topicId(), k -> new 
LinkedHashSet<>()).add(member)));
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            memberHashAssignment(partitions, List.of(member), newAssignment));
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = new 
LinkedHashSet<>(newAssignment.keySet());
+        Map<Uuid, List<TopicIdPartition>> unassignedPartitions = new 
HashMap<>();
+        targetPartitions.forEach(targetPartition -> {
+            if (!assignedPartitions.contains(targetPartition) && 
!currentAssignment.containsKey(targetPartition))
+                
unassignedPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
ArrayList<>()).add(targetPartition);
+        });
+
+        unassignedPartitions.keySet().forEach(unassignedTopic ->
+            
roundRobinAssignment(topicToMemberSubscription.get(unassignedTopic), 
unassignedPartitions.get(unassignedTopic), newAssignment));
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
member topic subscription in current assignment
+        // which is being subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> 
members.forEach(member -> {
+            if 
(topicToMemberSubscription.getOrDefault(targetPartition.topicId(), 
Collections.emptySet()).contains(member))
+                finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+        }));
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment groupAssignment(
+        Map<String, Set<TopicIdPartition>> assignmentByMember,
+        Collection<String> allGroupMembers
+    ) {
+        Map<String, MemberAssignment> members = new HashMap<>();
+        for (Map.Entry<String, Set<TopicIdPartition>> entry : 
assignmentByMember.entrySet()) {
+            Map<Uuid, Set<Integer>> targetPartitions = new HashMap<>();
+            entry.getValue().forEach(targetPartition -> 
targetPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
HashSet<>()).add(targetPartition.partitionId()));
+            members.put(entry.getKey(), new 
MemberAssignmentImpl(targetPartitions));
+        }
+        allGroupMembers.forEach(member -> {
+            if (!members.containsKey(member))
+                members.put(member, new MemberAssignmentImpl(new HashMap<>()));
+        });
+
         return new GroupAssignment(members);
     }
 
-    private Map<Uuid, Set<Integer>> computeTargetPartitions(
-        Set<Uuid> subscribeTopicIds,
+    /**
+     * This function updates assignment by hashing the member IDs of the 
members and maps the partitions assigned to the members based on the hash. This 
gives approximately even balance.
+     * @param targetPartitions - the subscribed topic partitions.
+     * @param memberIds - the member ids to which the topic partitions need to 
be assigned.
+     * @param assignment - the existing assignment by topic partition. We need 
to pass it as a parameter because this
+     *                   function would be called multiple times for 
heterogeneous assignment.
+     */
+    void memberHashAssignment(
+        List<TopicIdPartition> targetPartitions,
+        Collection<String> memberIds,
+        Map<TopicIdPartition, List<String>> assignment
+    ) {
+        if (!targetPartitions.isEmpty())
+            for (String memberId : memberIds) {
+                int topicPartitionIndex = Math.abs(memberId.hashCode() % 
targetPartitions.size());
+                TopicIdPartition topicPartition = 
targetPartitions.get(topicPartitionIndex);
+                assignment.computeIfAbsent(topicPartition, k -> new 
ArrayList<>()).add(memberId);
+            }
+    }
+
+    /**
+     * This functions assigns topic partitions to members by round-robin 
approach and updates the existing assignment.
+     * @param memberIds - the member ids to which the topic partitions need to 
be assigned.
+     * @param targetPartitions - the subscribed topic partitions.

Review Comment:
   This seems incorrect, it should be the partitions which needs assignment or 
something.



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java:
##########
@@ -67,42 +71,210 @@ private GroupAssignment assignHomogenous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Set<Uuid> subscribeTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
+        Set<Uuid> subscribedTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
             .subscribedTopicIds();
-        if (subscribeTopicIds.isEmpty())
-            return new GroupAssignment(Collections.emptyMap());
+        if (subscribedTopicIds.isEmpty())
+            return new GroupAssignment(Map.of());
 
-        Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
-            subscribeTopicIds, subscribedTopicDescriber);
+        // Subscribed topic partitions for the share group.
+        List<TopicIdPartition> targetPartitions = computeTargetPartitions(
+            subscribedTopicIds, subscribedTopicDescriber);
 
-        return new 
GroupAssignment(groupSpec.memberIds().stream().collect(Collectors.toMap(
-            Function.identity(), memberId -> new 
MemberAssignmentImpl(targetPartitions))));
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHomogeneous(groupSpec, subscribedTopicIds, 
targetPartitions, currentAssignment);
     }
 
     private GroupAssignment assignHeterogeneous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Map<String, MemberAssignment> members = new HashMap<>();
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription = 
new HashMap<>();
         for (String memberId : groupSpec.memberIds()) {
             MemberSubscription spec = groupSpec.memberSubscription(memberId);
             if (spec.subscribedTopicIds().isEmpty())
                 continue;
 
-            Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
+            // Subscribed topic partitions for the share group member.
+            List<TopicIdPartition> targetPartitions = computeTargetPartitions(
                 spec.subscribedTopicIds(), subscribedTopicDescriber);
+            memberToPartitionsSubscription.put(memberId, targetPartitions);
+        }
+
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHeterogeneous(groupSpec, 
memberToPartitionsSubscription, currentAssignment);
+    }
+
+    /**
+     * Get the current assignment by topic partitions.
+     * @param groupSpec - The group metadata specifications.
+     * @return the current assignment for subscribed topic partitions to 
memberIds.
+     */
+    private Map<TopicIdPartition, List<String>> currentAssignment(GroupSpec 
groupSpec) {
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
 
-            members.put(memberId, new MemberAssignmentImpl(targetPartitions));
+        for (String member : groupSpec.memberIds()) {
+            Map<Uuid, Set<Integer>> assignedTopicPartitions = 
groupSpec.memberAssignment(member).partitions();
+            assignedTopicPartitions.forEach((topicId, partitions) -> 
partitions.forEach(
+                partition -> assignment.computeIfAbsent(new 
TopicIdPartition(topicId, partition), k -> new ArrayList<>()).add(member)));
         }
+        return assignment;
+    }
+
+    private GroupAssignment newAssignmentHomogeneous(
+        GroupSpec groupSpec,
+        Set<Uuid> subscribedTopicIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberHashAssignment(targetPartitions, groupSpec.memberIds(), 
newAssignment);
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = newAssignment.keySet();
+        List<TopicIdPartition> unassignedPartitions = targetPartitions.stream()
+            .filter(targetPartition -> 
!assignedPartitions.contains(targetPartition))
+            .filter(targetPartition -> 
!currentAssignment.containsKey(targetPartition))
+            .toList();
+
+        roundRobinAssignment(groupSpec.memberIds(), unassignedPartitions, 
newAssignment);
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
topics in current assignment that are also being
+        // subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> {
+            if (subscribedTopicIds.contains(targetPartition.topicId()))
+                members.forEach(member -> {
+                    if (groupSpec.memberIds().contains(member))
+                        finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+                });
+        });
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment newAssignmentHeterogeneous(
+        GroupSpec groupSpec,
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        // Exhaustive set of all subscribed topic partitions.
+        Set<TopicIdPartition> targetPartitions = new LinkedHashSet<>();
+        
memberToPartitionsSubscription.values().forEach(targetPartitions::addAll);
+
+        // Create a map for topic to members subscription.
+        Map<Uuid, Set<String>> topicToMemberSubscription = new HashMap<>();
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            partitions.forEach(partition -> 
topicToMemberSubscription.computeIfAbsent(partition.topicId(), k -> new 
LinkedHashSet<>()).add(member)));
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            memberHashAssignment(partitions, List.of(member), newAssignment));
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = new 
LinkedHashSet<>(newAssignment.keySet());
+        Map<Uuid, List<TopicIdPartition>> unassignedPartitions = new 
HashMap<>();
+        targetPartitions.forEach(targetPartition -> {
+            if (!assignedPartitions.contains(targetPartition) && 
!currentAssignment.containsKey(targetPartition))
+                
unassignedPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
ArrayList<>()).add(targetPartition);
+        });
+
+        unassignedPartitions.keySet().forEach(unassignedTopic ->
+            
roundRobinAssignment(topicToMemberSubscription.get(unassignedTopic), 
unassignedPartitions.get(unassignedTopic), newAssignment));
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
member topic subscription in current assignment
+        // which is being subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> 
members.forEach(member -> {

Review Comment:
   ```suggestion
           currentAssignment.forEach((topicIdPartition, members) -> 
members.forEach(member -> {
   ```



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java:
##########
@@ -67,42 +71,210 @@ private GroupAssignment assignHomogenous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Set<Uuid> subscribeTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
+        Set<Uuid> subscribedTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
             .subscribedTopicIds();
-        if (subscribeTopicIds.isEmpty())
-            return new GroupAssignment(Collections.emptyMap());
+        if (subscribedTopicIds.isEmpty())
+            return new GroupAssignment(Map.of());
 
-        Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
-            subscribeTopicIds, subscribedTopicDescriber);
+        // Subscribed topic partitions for the share group.
+        List<TopicIdPartition> targetPartitions = computeTargetPartitions(
+            subscribedTopicIds, subscribedTopicDescriber);
 
-        return new 
GroupAssignment(groupSpec.memberIds().stream().collect(Collectors.toMap(
-            Function.identity(), memberId -> new 
MemberAssignmentImpl(targetPartitions))));
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHomogeneous(groupSpec, subscribedTopicIds, 
targetPartitions, currentAssignment);
     }
 
     private GroupAssignment assignHeterogeneous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Map<String, MemberAssignment> members = new HashMap<>();
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription = 
new HashMap<>();
         for (String memberId : groupSpec.memberIds()) {
             MemberSubscription spec = groupSpec.memberSubscription(memberId);
             if (spec.subscribedTopicIds().isEmpty())
                 continue;
 
-            Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
+            // Subscribed topic partitions for the share group member.
+            List<TopicIdPartition> targetPartitions = computeTargetPartitions(
                 spec.subscribedTopicIds(), subscribedTopicDescriber);
+            memberToPartitionsSubscription.put(memberId, targetPartitions);
+        }
+
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHeterogeneous(groupSpec, 
memberToPartitionsSubscription, currentAssignment);
+    }
+
+    /**
+     * Get the current assignment by topic partitions.
+     * @param groupSpec - The group metadata specifications.
+     * @return the current assignment for subscribed topic partitions to 
memberIds.
+     */
+    private Map<TopicIdPartition, List<String>> currentAssignment(GroupSpec 
groupSpec) {
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
 
-            members.put(memberId, new MemberAssignmentImpl(targetPartitions));
+        for (String member : groupSpec.memberIds()) {
+            Map<Uuid, Set<Integer>> assignedTopicPartitions = 
groupSpec.memberAssignment(member).partitions();
+            assignedTopicPartitions.forEach((topicId, partitions) -> 
partitions.forEach(
+                partition -> assignment.computeIfAbsent(new 
TopicIdPartition(topicId, partition), k -> new ArrayList<>()).add(member)));
         }
+        return assignment;
+    }
+
+    private GroupAssignment newAssignmentHomogeneous(
+        GroupSpec groupSpec,
+        Set<Uuid> subscribedTopicIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberHashAssignment(targetPartitions, groupSpec.memberIds(), 
newAssignment);
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = newAssignment.keySet();
+        List<TopicIdPartition> unassignedPartitions = targetPartitions.stream()
+            .filter(targetPartition -> 
!assignedPartitions.contains(targetPartition))
+            .filter(targetPartition -> 
!currentAssignment.containsKey(targetPartition))
+            .toList();
+
+        roundRobinAssignment(groupSpec.memberIds(), unassignedPartitions, 
newAssignment);
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
topics in current assignment that are also being
+        // subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> {
+            if (subscribedTopicIds.contains(targetPartition.topicId()))
+                members.forEach(member -> {
+                    if (groupSpec.memberIds().contains(member))
+                        finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+                });
+        });
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment newAssignmentHeterogeneous(
+        GroupSpec groupSpec,
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        // Exhaustive set of all subscribed topic partitions.
+        Set<TopicIdPartition> targetPartitions = new LinkedHashSet<>();
+        
memberToPartitionsSubscription.values().forEach(targetPartitions::addAll);
+
+        // Create a map for topic to members subscription.
+        Map<Uuid, Set<String>> topicToMemberSubscription = new HashMap<>();
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            partitions.forEach(partition -> 
topicToMemberSubscription.computeIfAbsent(partition.topicId(), k -> new 
LinkedHashSet<>()).add(member)));
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            memberHashAssignment(partitions, List.of(member), newAssignment));
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = new 
LinkedHashSet<>(newAssignment.keySet());
+        Map<Uuid, List<TopicIdPartition>> unassignedPartitions = new 
HashMap<>();
+        targetPartitions.forEach(targetPartition -> {
+            if (!assignedPartitions.contains(targetPartition) && 
!currentAssignment.containsKey(targetPartition))
+                
unassignedPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
ArrayList<>()).add(targetPartition);
+        });
+
+        unassignedPartitions.keySet().forEach(unassignedTopic ->
+            
roundRobinAssignment(topicToMemberSubscription.get(unassignedTopic), 
unassignedPartitions.get(unassignedTopic), newAssignment));
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
member topic subscription in current assignment
+        // which is being subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> 
members.forEach(member -> {
+            if 
(topicToMemberSubscription.getOrDefault(targetPartition.topicId(), 
Collections.emptySet()).contains(member))
+                finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+        }));
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment groupAssignment(
+        Map<String, Set<TopicIdPartition>> assignmentByMember,
+        Collection<String> allGroupMembers
+    ) {
+        Map<String, MemberAssignment> members = new HashMap<>();
+        for (Map.Entry<String, Set<TopicIdPartition>> entry : 
assignmentByMember.entrySet()) {
+            Map<Uuid, Set<Integer>> targetPartitions = new HashMap<>();
+            entry.getValue().forEach(targetPartition -> 
targetPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
HashSet<>()).add(targetPartition.partitionId()));
+            members.put(entry.getKey(), new 
MemberAssignmentImpl(targetPartitions));
+        }
+        allGroupMembers.forEach(member -> {
+            if (!members.containsKey(member))
+                members.put(member, new MemberAssignmentImpl(new HashMap<>()));
+        });
+
         return new GroupAssignment(members);
     }
 
-    private Map<Uuid, Set<Integer>> computeTargetPartitions(
-        Set<Uuid> subscribeTopicIds,
+    /**
+     * This function updates assignment by hashing the member IDs of the 
members and maps the partitions assigned to the members based on the hash. This 
gives approximately even balance.
+     * @param targetPartitions - the subscribed topic partitions.
+     * @param memberIds - the member ids to which the topic partitions need to 
be assigned.
+     * @param assignment - the existing assignment by topic partition. We need 
to pass it as a parameter because this
+     *                   function would be called multiple times for 
heterogeneous assignment.
+     */
+    void memberHashAssignment(
+        List<TopicIdPartition> targetPartitions,
+        Collection<String> memberIds,
+        Map<TopicIdPartition, List<String>> assignment
+    ) {
+        if (!targetPartitions.isEmpty())
+            for (String memberId : memberIds) {
+                int topicPartitionIndex = Math.abs(memberId.hashCode() % 
targetPartitions.size());
+                TopicIdPartition topicPartition = 
targetPartitions.get(topicPartitionIndex);
+                assignment.computeIfAbsent(topicPartition, k -> new 
ArrayList<>()).add(memberId);
+            }
+    }
+
+    /**
+     * This functions assigns topic partitions to members by round-robin 
approach and updates the existing assignment.
+     * @param memberIds - the member ids to which the topic partitions need to 
be assigned.
+     * @param targetPartitions - the subscribed topic partitions.
+     * @param assignment - the existing assignment by topic partition.
+     */
+    void roundRobinAssignment(
+        Collection<String> memberIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> assignment
+    ) {
+        // We iterate through the target partitions and assign a memberId to 
them. In case we run out of members (members < targetPartitions),
+        // we again start from the starting index of memberIds.
+        Iterator<String> memberIdIterator = memberIds.iterator();
+        for (TopicIdPartition targetPartition : targetPartitions) {
+            if (!memberIdIterator.hasNext()) {
+                memberIdIterator = memberIds.iterator();
+            }
+            String member = memberIdIterator.next();

Review Comment:
   ```suggestion
               String memberId = memberIdIterator.next();
   ```



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java:
##########
@@ -67,42 +71,210 @@ private GroupAssignment assignHomogenous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Set<Uuid> subscribeTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
+        Set<Uuid> subscribedTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
             .subscribedTopicIds();
-        if (subscribeTopicIds.isEmpty())
-            return new GroupAssignment(Collections.emptyMap());
+        if (subscribedTopicIds.isEmpty())
+            return new GroupAssignment(Map.of());
 
-        Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
-            subscribeTopicIds, subscribedTopicDescriber);
+        // Subscribed topic partitions for the share group.
+        List<TopicIdPartition> targetPartitions = computeTargetPartitions(
+            subscribedTopicIds, subscribedTopicDescriber);
 
-        return new 
GroupAssignment(groupSpec.memberIds().stream().collect(Collectors.toMap(
-            Function.identity(), memberId -> new 
MemberAssignmentImpl(targetPartitions))));
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHomogeneous(groupSpec, subscribedTopicIds, 
targetPartitions, currentAssignment);
     }
 
     private GroupAssignment assignHeterogeneous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Map<String, MemberAssignment> members = new HashMap<>();
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription = 
new HashMap<>();
         for (String memberId : groupSpec.memberIds()) {
             MemberSubscription spec = groupSpec.memberSubscription(memberId);
             if (spec.subscribedTopicIds().isEmpty())
                 continue;
 
-            Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
+            // Subscribed topic partitions for the share group member.
+            List<TopicIdPartition> targetPartitions = computeTargetPartitions(
                 spec.subscribedTopicIds(), subscribedTopicDescriber);
+            memberToPartitionsSubscription.put(memberId, targetPartitions);
+        }
+
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHeterogeneous(groupSpec, 
memberToPartitionsSubscription, currentAssignment);
+    }
+
+    /**
+     * Get the current assignment by topic partitions.
+     * @param groupSpec - The group metadata specifications.
+     * @return the current assignment for subscribed topic partitions to 
memberIds.
+     */
+    private Map<TopicIdPartition, List<String>> currentAssignment(GroupSpec 
groupSpec) {
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
 
-            members.put(memberId, new MemberAssignmentImpl(targetPartitions));
+        for (String member : groupSpec.memberIds()) {
+            Map<Uuid, Set<Integer>> assignedTopicPartitions = 
groupSpec.memberAssignment(member).partitions();
+            assignedTopicPartitions.forEach((topicId, partitions) -> 
partitions.forEach(
+                partition -> assignment.computeIfAbsent(new 
TopicIdPartition(topicId, partition), k -> new ArrayList<>()).add(member)));
         }
+        return assignment;
+    }
+
+    private GroupAssignment newAssignmentHomogeneous(
+        GroupSpec groupSpec,
+        Set<Uuid> subscribedTopicIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberHashAssignment(targetPartitions, groupSpec.memberIds(), 
newAssignment);
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = newAssignment.keySet();
+        List<TopicIdPartition> unassignedPartitions = targetPartitions.stream()
+            .filter(targetPartition -> 
!assignedPartitions.contains(targetPartition))
+            .filter(targetPartition -> 
!currentAssignment.containsKey(targetPartition))
+            .toList();
+
+        roundRobinAssignment(groupSpec.memberIds(), unassignedPartitions, 
newAssignment);
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
topics in current assignment that are also being
+        // subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> {
+            if (subscribedTopicIds.contains(targetPartition.topicId()))
+                members.forEach(member -> {
+                    if (groupSpec.memberIds().contains(member))
+                        finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+                });
+        });
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));

Review Comment:
   Is it aligned with the approach defiend in step 3 of the KIP?



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignorTest.java:
##########
@@ -320,12 +365,377 @@ public void 
testAssignWithOneMemberNoAssignedTopicHeterogeneous() {
         Map<String, Map<Uuid, Set<Integer>>> expectedAssignment = new 
HashMap<>();
         expectedAssignment.put(MEMBER_A, mkAssignment(
             mkTopicAssignment(TOPIC_1_UUID, 0, 1, 2),
-            mkTopicAssignment(TOPIC_2_UUID, 0, 1)
-        ));
+            mkTopicAssignment(TOPIC_2_UUID, 0, 1)));
+        expectedAssignment.put(MEMBER_B, mkAssignment());
 
+        // T1: 3 partitions + T2: 2 partitions = 5 partitions
+        assertEveryPartitionGetsAssignment(5, computedAssignment);
         assertAssignment(expectedAssignment, computedAssignment);
     }
 
+    @Test
+    public void testMemberHashAssignment() {
+        // hashcode for "member1" is 948881623.
+        String member1 = "member1";
+        // hashcode for "member2" is 948881624.
+        String member2 = "member2";
+        // hashcode for "member3" is 948881625.
+        String member3 = "member3";
+        // hashcode for "member4" is 948881626.
+        String member4 = "member4";
+        // hashcode for "AaAaAaAa" is -540425984 to test with negative 
hashcode.
+        String member5 = "AaAaAaAa";
+        List<String> members = Arrays.asList(member1, member2, member3, 
member4, member5);
+
+        TopicIdPartition partition1 = new TopicIdPartition(TOPIC_1_UUID, 0);
+        TopicIdPartition partition2 = new TopicIdPartition(TOPIC_2_UUID, 0);
+        TopicIdPartition partition3 = new TopicIdPartition(TOPIC_3_UUID, 0);
+        List<TopicIdPartition> partitions = Arrays.asList(partition1, 
partition2, partition3);
+
+        Map<TopicIdPartition, List<String>> computedAssignment = new 
HashMap<>();
+        assignor.memberHashAssignment(partitions, members, computedAssignment);
+
+        Map<TopicIdPartition, List<String>> expectedAssignment = new 
HashMap<>();
+        expectedAssignment.put(partition1, List.of(member3));
+        expectedAssignment.put(partition2, Arrays.asList(member1, member4));
+        expectedAssignment.put(partition3, Arrays.asList(member2, member5));
+        assertAssignment(expectedAssignment, computedAssignment);
+    }
+
+    @Test
+    public void testRoundRobinAssignment() {
+        String member1 = "member1";
+        String member2 = "member2";
+        List<String> members = Arrays.asList(member1, member2);
+        TopicIdPartition partition1 = new TopicIdPartition(TOPIC_1_UUID, 0);
+        TopicIdPartition partition2 = new TopicIdPartition(TOPIC_2_UUID, 0);
+        TopicIdPartition partition3 = new TopicIdPartition(TOPIC_3_UUID, 0);
+        TopicIdPartition partition4 = new TopicIdPartition(TOPIC_4_UUID, 0);
+        List<TopicIdPartition> unassignedPartitions = 
Arrays.asList(partition2, partition3, partition4);
+
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
+        assignment.put(partition1, List.of(member1));
+
+        assignor.roundRobinAssignment(members, unassignedPartitions, 
assignment);
+        Map<TopicIdPartition, List<String>> expectedAssignment = new 
HashMap<>();
+        expectedAssignment.put(partition1, List.of(member1));
+        expectedAssignment.put(partition2, List.of(member1));
+        expectedAssignment.put(partition3, List.of(member2));
+        expectedAssignment.put(partition4, List.of(member1));

Review Comment:
   nit: can use Map.of(.....)



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java:
##########
@@ -67,42 +71,210 @@ private GroupAssignment assignHomogenous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Set<Uuid> subscribeTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
+        Set<Uuid> subscribedTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
             .subscribedTopicIds();
-        if (subscribeTopicIds.isEmpty())
-            return new GroupAssignment(Collections.emptyMap());
+        if (subscribedTopicIds.isEmpty())
+            return new GroupAssignment(Map.of());
 
-        Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
-            subscribeTopicIds, subscribedTopicDescriber);
+        // Subscribed topic partitions for the share group.
+        List<TopicIdPartition> targetPartitions = computeTargetPartitions(
+            subscribedTopicIds, subscribedTopicDescriber);
 
-        return new 
GroupAssignment(groupSpec.memberIds().stream().collect(Collectors.toMap(
-            Function.identity(), memberId -> new 
MemberAssignmentImpl(targetPartitions))));
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHomogeneous(groupSpec, subscribedTopicIds, 
targetPartitions, currentAssignment);
     }
 
     private GroupAssignment assignHeterogeneous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Map<String, MemberAssignment> members = new HashMap<>();
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription = 
new HashMap<>();
         for (String memberId : groupSpec.memberIds()) {
             MemberSubscription spec = groupSpec.memberSubscription(memberId);
             if (spec.subscribedTopicIds().isEmpty())
                 continue;
 
-            Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
+            // Subscribed topic partitions for the share group member.
+            List<TopicIdPartition> targetPartitions = computeTargetPartitions(
                 spec.subscribedTopicIds(), subscribedTopicDescriber);
+            memberToPartitionsSubscription.put(memberId, targetPartitions);
+        }
+
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHeterogeneous(groupSpec, 
memberToPartitionsSubscription, currentAssignment);
+    }
+
+    /**
+     * Get the current assignment by topic partitions.
+     * @param groupSpec - The group metadata specifications.
+     * @return the current assignment for subscribed topic partitions to 
memberIds.
+     */
+    private Map<TopicIdPartition, List<String>> currentAssignment(GroupSpec 
groupSpec) {
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
 
-            members.put(memberId, new MemberAssignmentImpl(targetPartitions));
+        for (String member : groupSpec.memberIds()) {
+            Map<Uuid, Set<Integer>> assignedTopicPartitions = 
groupSpec.memberAssignment(member).partitions();
+            assignedTopicPartitions.forEach((topicId, partitions) -> 
partitions.forEach(
+                partition -> assignment.computeIfAbsent(new 
TopicIdPartition(topicId, partition), k -> new ArrayList<>()).add(member)));
         }
+        return assignment;
+    }
+
+    private GroupAssignment newAssignmentHomogeneous(
+        GroupSpec groupSpec,
+        Set<Uuid> subscribedTopicIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberHashAssignment(targetPartitions, groupSpec.memberIds(), 
newAssignment);
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = newAssignment.keySet();
+        List<TopicIdPartition> unassignedPartitions = targetPartitions.stream()
+            .filter(targetPartition -> 
!assignedPartitions.contains(targetPartition))
+            .filter(targetPartition -> 
!currentAssignment.containsKey(targetPartition))
+            .toList();
+
+        roundRobinAssignment(groupSpec.memberIds(), unassignedPartitions, 
newAssignment);
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
topics in current assignment that are also being
+        // subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> {
+            if (subscribedTopicIds.contains(targetPartition.topicId()))
+                members.forEach(member -> {
+                    if (groupSpec.memberIds().contains(member))
+                        finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+                });
+        });
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment newAssignmentHeterogeneous(
+        GroupSpec groupSpec,
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        // Exhaustive set of all subscribed topic partitions.
+        Set<TopicIdPartition> targetPartitions = new LinkedHashSet<>();
+        
memberToPartitionsSubscription.values().forEach(targetPartitions::addAll);
+
+        // Create a map for topic to members subscription.
+        Map<Uuid, Set<String>> topicToMemberSubscription = new HashMap<>();
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            partitions.forEach(partition -> 
topicToMemberSubscription.computeIfAbsent(partition.topicId(), k -> new 
LinkedHashSet<>()).add(member)));
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            memberHashAssignment(partitions, List.of(member), newAssignment));
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = new 
LinkedHashSet<>(newAssignment.keySet());
+        Map<Uuid, List<TopicIdPartition>> unassignedPartitions = new 
HashMap<>();
+        targetPartitions.forEach(targetPartition -> {
+            if (!assignedPartitions.contains(targetPartition) && 
!currentAssignment.containsKey(targetPartition))
+                
unassignedPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
ArrayList<>()).add(targetPartition);
+        });
+
+        unassignedPartitions.keySet().forEach(unassignedTopic ->
+            
roundRobinAssignment(topicToMemberSubscription.get(unassignedTopic), 
unassignedPartitions.get(unassignedTopic), newAssignment));
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
member topic subscription in current assignment
+        // which is being subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> 
members.forEach(member -> {
+            if 
(topicToMemberSubscription.getOrDefault(targetPartition.topicId(), 
Collections.emptySet()).contains(member))
+                finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+        }));
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));

Review Comment:
   Is this as per the KIP or a changed approach to have more sharing? If 
changed then can you please write details in comment for now.



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignorTest.java:
##########
@@ -320,12 +365,377 @@ public void 
testAssignWithOneMemberNoAssignedTopicHeterogeneous() {
         Map<String, Map<Uuid, Set<Integer>>> expectedAssignment = new 
HashMap<>();
         expectedAssignment.put(MEMBER_A, mkAssignment(
             mkTopicAssignment(TOPIC_1_UUID, 0, 1, 2),
-            mkTopicAssignment(TOPIC_2_UUID, 0, 1)
-        ));
+            mkTopicAssignment(TOPIC_2_UUID, 0, 1)));
+        expectedAssignment.put(MEMBER_B, mkAssignment());
 
+        // T1: 3 partitions + T2: 2 partitions = 5 partitions
+        assertEveryPartitionGetsAssignment(5, computedAssignment);
         assertAssignment(expectedAssignment, computedAssignment);
     }
 
+    @Test
+    public void testMemberHashAssignment() {
+        // hashcode for "member1" is 948881623.
+        String member1 = "member1";
+        // hashcode for "member2" is 948881624.
+        String member2 = "member2";
+        // hashcode for "member3" is 948881625.
+        String member3 = "member3";
+        // hashcode for "member4" is 948881626.
+        String member4 = "member4";
+        // hashcode for "AaAaAaAa" is -540425984 to test with negative 
hashcode.
+        String member5 = "AaAaAaAa";
+        List<String> members = Arrays.asList(member1, member2, member3, 
member4, member5);

Review Comment:
   ```suggestion
           List<String> members = List.of(member1, member2, member3, member4, 
member5);
   ```



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java:
##########
@@ -67,42 +71,210 @@ private GroupAssignment assignHomogenous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Set<Uuid> subscribeTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
+        Set<Uuid> subscribedTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
             .subscribedTopicIds();
-        if (subscribeTopicIds.isEmpty())
-            return new GroupAssignment(Collections.emptyMap());
+        if (subscribedTopicIds.isEmpty())
+            return new GroupAssignment(Map.of());
 
-        Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
-            subscribeTopicIds, subscribedTopicDescriber);
+        // Subscribed topic partitions for the share group.
+        List<TopicIdPartition> targetPartitions = computeTargetPartitions(
+            subscribedTopicIds, subscribedTopicDescriber);
 
-        return new 
GroupAssignment(groupSpec.memberIds().stream().collect(Collectors.toMap(
-            Function.identity(), memberId -> new 
MemberAssignmentImpl(targetPartitions))));
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHomogeneous(groupSpec, subscribedTopicIds, 
targetPartitions, currentAssignment);
     }
 
     private GroupAssignment assignHeterogeneous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Map<String, MemberAssignment> members = new HashMap<>();
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription = 
new HashMap<>();
         for (String memberId : groupSpec.memberIds()) {
             MemberSubscription spec = groupSpec.memberSubscription(memberId);
             if (spec.subscribedTopicIds().isEmpty())
                 continue;
 
-            Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
+            // Subscribed topic partitions for the share group member.
+            List<TopicIdPartition> targetPartitions = computeTargetPartitions(
                 spec.subscribedTopicIds(), subscribedTopicDescriber);
+            memberToPartitionsSubscription.put(memberId, targetPartitions);
+        }
+
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHeterogeneous(groupSpec, 
memberToPartitionsSubscription, currentAssignment);
+    }
+
+    /**
+     * Get the current assignment by topic partitions.
+     * @param groupSpec - The group metadata specifications.
+     * @return the current assignment for subscribed topic partitions to 
memberIds.
+     */
+    private Map<TopicIdPartition, List<String>> currentAssignment(GroupSpec 
groupSpec) {
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
 
-            members.put(memberId, new MemberAssignmentImpl(targetPartitions));
+        for (String member : groupSpec.memberIds()) {
+            Map<Uuid, Set<Integer>> assignedTopicPartitions = 
groupSpec.memberAssignment(member).partitions();
+            assignedTopicPartitions.forEach((topicId, partitions) -> 
partitions.forEach(
+                partition -> assignment.computeIfAbsent(new 
TopicIdPartition(topicId, partition), k -> new ArrayList<>()).add(member)));
         }
+        return assignment;
+    }
+
+    private GroupAssignment newAssignmentHomogeneous(
+        GroupSpec groupSpec,
+        Set<Uuid> subscribedTopicIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberHashAssignment(targetPartitions, groupSpec.memberIds(), 
newAssignment);
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = newAssignment.keySet();
+        List<TopicIdPartition> unassignedPartitions = targetPartitions.stream()
+            .filter(targetPartition -> 
!assignedPartitions.contains(targetPartition))
+            .filter(targetPartition -> 
!currentAssignment.containsKey(targetPartition))
+            .toList();
+
+        roundRobinAssignment(groupSpec.memberIds(), unassignedPartitions, 
newAssignment);
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
topics in current assignment that are also being
+        // subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> {
+            if (subscribedTopicIds.contains(targetPartition.topicId()))
+                members.forEach(member -> {
+                    if (groupSpec.memberIds().contains(member))
+                        finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+                });
+        });
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment newAssignmentHeterogeneous(
+        GroupSpec groupSpec,
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        // Exhaustive set of all subscribed topic partitions.
+        Set<TopicIdPartition> targetPartitions = new LinkedHashSet<>();
+        
memberToPartitionsSubscription.values().forEach(targetPartitions::addAll);
+
+        // Create a map for topic to members subscription.
+        Map<Uuid, Set<String>> topicToMemberSubscription = new HashMap<>();
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            partitions.forEach(partition -> 
topicToMemberSubscription.computeIfAbsent(partition.topicId(), k -> new 
LinkedHashSet<>()).add(member)));
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            memberHashAssignment(partitions, List.of(member), newAssignment));
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = new 
LinkedHashSet<>(newAssignment.keySet());
+        Map<Uuid, List<TopicIdPartition>> unassignedPartitions = new 
HashMap<>();
+        targetPartitions.forEach(targetPartition -> {
+            if (!assignedPartitions.contains(targetPartition) && 
!currentAssignment.containsKey(targetPartition))
+                
unassignedPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
ArrayList<>()).add(targetPartition);
+        });
+
+        unassignedPartitions.keySet().forEach(unassignedTopic ->
+            
roundRobinAssignment(topicToMemberSubscription.get(unassignedTopic), 
unassignedPartitions.get(unassignedTopic), newAssignment));
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
member topic subscription in current assignment
+        // which is being subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> 
members.forEach(member -> {
+            if 
(topicToMemberSubscription.getOrDefault(targetPartition.topicId(), 
Collections.emptySet()).contains(member))
+                finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+        }));
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment groupAssignment(
+        Map<String, Set<TopicIdPartition>> assignmentByMember,
+        Collection<String> allGroupMembers
+    ) {
+        Map<String, MemberAssignment> members = new HashMap<>();
+        for (Map.Entry<String, Set<TopicIdPartition>> entry : 
assignmentByMember.entrySet()) {
+            Map<Uuid, Set<Integer>> targetPartitions = new HashMap<>();
+            entry.getValue().forEach(targetPartition -> 
targetPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
HashSet<>()).add(targetPartition.partitionId()));
+            members.put(entry.getKey(), new 
MemberAssignmentImpl(targetPartitions));
+        }
+        allGroupMembers.forEach(member -> {
+            if (!members.containsKey(member))
+                members.put(member, new MemberAssignmentImpl(new HashMap<>()));
+        });
+
         return new GroupAssignment(members);
     }
 
-    private Map<Uuid, Set<Integer>> computeTargetPartitions(
-        Set<Uuid> subscribeTopicIds,
+    /**
+     * This function updates assignment by hashing the member IDs of the 
members and maps the partitions assigned to the members based on the hash. This 
gives approximately even balance.
+     * @param targetPartitions - the subscribed topic partitions.
+     * @param memberIds - the member ids to which the topic partitions need to 
be assigned.
+     * @param assignment - the existing assignment by topic partition. We need 
to pass it as a parameter because this
+     *                   function would be called multiple times for 
heterogeneous assignment.
+     */
+    void memberHashAssignment(
+        List<TopicIdPartition> targetPartitions,
+        Collection<String> memberIds,
+        Map<TopicIdPartition, List<String>> assignment
+    ) {
+        if (!targetPartitions.isEmpty())
+            for (String memberId : memberIds) {
+                int topicPartitionIndex = Math.abs(memberId.hashCode() % 
targetPartitions.size());
+                TopicIdPartition topicPartition = 
targetPartitions.get(topicPartitionIndex);
+                assignment.computeIfAbsent(topicPartition, k -> new 
ArrayList<>()).add(memberId);
+            }
+    }
+
+    /**
+     * This functions assigns topic partitions to members by round-robin 
approach and updates the existing assignment.
+     * @param memberIds - the member ids to which the topic partitions need to 
be assigned.
+     * @param targetPartitions - the subscribed topic partitions.
+     * @param assignment - the existing assignment by topic partition.
+     */
+    void roundRobinAssignment(
+        Collection<String> memberIds,
+        List<TopicIdPartition> targetPartitions,

Review Comment:
   targetPartitions seems confusing here.
   ```suggestion
           List<TopicIdPartition> unassignedPartitions/topicIdPartitions,
   ```



##########
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignorTest.java:
##########
@@ -320,12 +365,377 @@ public void 
testAssignWithOneMemberNoAssignedTopicHeterogeneous() {
         Map<String, Map<Uuid, Set<Integer>>> expectedAssignment = new 
HashMap<>();
         expectedAssignment.put(MEMBER_A, mkAssignment(
             mkTopicAssignment(TOPIC_1_UUID, 0, 1, 2),
-            mkTopicAssignment(TOPIC_2_UUID, 0, 1)
-        ));
+            mkTopicAssignment(TOPIC_2_UUID, 0, 1)));
+        expectedAssignment.put(MEMBER_B, mkAssignment());
 
+        // T1: 3 partitions + T2: 2 partitions = 5 partitions
+        assertEveryPartitionGetsAssignment(5, computedAssignment);
         assertAssignment(expectedAssignment, computedAssignment);
     }
 
+    @Test
+    public void testMemberHashAssignment() {
+        // hashcode for "member1" is 948881623.
+        String member1 = "member1";
+        // hashcode for "member2" is 948881624.
+        String member2 = "member2";
+        // hashcode for "member3" is 948881625.
+        String member3 = "member3";
+        // hashcode for "member4" is 948881626.
+        String member4 = "member4";
+        // hashcode for "AaAaAaAa" is -540425984 to test with negative 
hashcode.
+        String member5 = "AaAaAaAa";
+        List<String> members = Arrays.asList(member1, member2, member3, 
member4, member5);
+
+        TopicIdPartition partition1 = new TopicIdPartition(TOPIC_1_UUID, 0);
+        TopicIdPartition partition2 = new TopicIdPartition(TOPIC_2_UUID, 0);
+        TopicIdPartition partition3 = new TopicIdPartition(TOPIC_3_UUID, 0);
+        List<TopicIdPartition> partitions = Arrays.asList(partition1, 
partition2, partition3);
+
+        Map<TopicIdPartition, List<String>> computedAssignment = new 
HashMap<>();
+        assignor.memberHashAssignment(partitions, members, computedAssignment);
+
+        Map<TopicIdPartition, List<String>> expectedAssignment = new 
HashMap<>();
+        expectedAssignment.put(partition1, List.of(member3));
+        expectedAssignment.put(partition2, Arrays.asList(member1, member4));

Review Comment:
   Why not List.of as in line above?



##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignor.java:
##########
@@ -67,42 +71,210 @@ private GroupAssignment assignHomogenous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Set<Uuid> subscribeTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
+        Set<Uuid> subscribedTopicIds = 
groupSpec.memberSubscription(groupSpec.memberIds().iterator().next())
             .subscribedTopicIds();
-        if (subscribeTopicIds.isEmpty())
-            return new GroupAssignment(Collections.emptyMap());
+        if (subscribedTopicIds.isEmpty())
+            return new GroupAssignment(Map.of());
 
-        Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
-            subscribeTopicIds, subscribedTopicDescriber);
+        // Subscribed topic partitions for the share group.
+        List<TopicIdPartition> targetPartitions = computeTargetPartitions(
+            subscribedTopicIds, subscribedTopicDescriber);
 
-        return new 
GroupAssignment(groupSpec.memberIds().stream().collect(Collectors.toMap(
-            Function.identity(), memberId -> new 
MemberAssignmentImpl(targetPartitions))));
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHomogeneous(groupSpec, subscribedTopicIds, 
targetPartitions, currentAssignment);
     }
 
     private GroupAssignment assignHeterogeneous(
         GroupSpec groupSpec,
         SubscribedTopicDescriber subscribedTopicDescriber
     ) {
-        Map<String, MemberAssignment> members = new HashMap<>();
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription = 
new HashMap<>();
         for (String memberId : groupSpec.memberIds()) {
             MemberSubscription spec = groupSpec.memberSubscription(memberId);
             if (spec.subscribedTopicIds().isEmpty())
                 continue;
 
-            Map<Uuid, Set<Integer>> targetPartitions = computeTargetPartitions(
+            // Subscribed topic partitions for the share group member.
+            List<TopicIdPartition> targetPartitions = computeTargetPartitions(
                 spec.subscribedTopicIds(), subscribedTopicDescriber);
+            memberToPartitionsSubscription.put(memberId, targetPartitions);
+        }
+
+        // The current assignment from topic partition to members.
+        Map<TopicIdPartition, List<String>> currentAssignment = 
currentAssignment(groupSpec);
+        return newAssignmentHeterogeneous(groupSpec, 
memberToPartitionsSubscription, currentAssignment);
+    }
+
+    /**
+     * Get the current assignment by topic partitions.
+     * @param groupSpec - The group metadata specifications.
+     * @return the current assignment for subscribed topic partitions to 
memberIds.
+     */
+    private Map<TopicIdPartition, List<String>> currentAssignment(GroupSpec 
groupSpec) {
+        Map<TopicIdPartition, List<String>> assignment = new HashMap<>();
 
-            members.put(memberId, new MemberAssignmentImpl(targetPartitions));
+        for (String member : groupSpec.memberIds()) {
+            Map<Uuid, Set<Integer>> assignedTopicPartitions = 
groupSpec.memberAssignment(member).partitions();
+            assignedTopicPartitions.forEach((topicId, partitions) -> 
partitions.forEach(
+                partition -> assignment.computeIfAbsent(new 
TopicIdPartition(topicId, partition), k -> new ArrayList<>()).add(member)));
         }
+        return assignment;
+    }
+
+    private GroupAssignment newAssignmentHomogeneous(
+        GroupSpec groupSpec,
+        Set<Uuid> subscribedTopicIds,
+        List<TopicIdPartition> targetPartitions,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberHashAssignment(targetPartitions, groupSpec.memberIds(), 
newAssignment);
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = newAssignment.keySet();
+        List<TopicIdPartition> unassignedPartitions = targetPartitions.stream()
+            .filter(targetPartition -> 
!assignedPartitions.contains(targetPartition))
+            .filter(targetPartition -> 
!currentAssignment.containsKey(targetPartition))
+            .toList();
+
+        roundRobinAssignment(groupSpec.memberIds(), unassignedPartitions, 
newAssignment);
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
topics in current assignment that are also being
+        // subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> {
+            if (subscribedTopicIds.contains(targetPartition.topicId()))
+                members.forEach(member -> {
+                    if (groupSpec.memberIds().contains(member))
+                        finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+                });
+        });
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment newAssignmentHeterogeneous(
+        GroupSpec groupSpec,
+        Map<String, List<TopicIdPartition>> memberToPartitionsSubscription,
+        Map<TopicIdPartition, List<String>> currentAssignment
+    ) {
+
+        // Exhaustive set of all subscribed topic partitions.
+        Set<TopicIdPartition> targetPartitions = new LinkedHashSet<>();
+        
memberToPartitionsSubscription.values().forEach(targetPartitions::addAll);
+
+        // Create a map for topic to members subscription.
+        Map<Uuid, Set<String>> topicToMemberSubscription = new HashMap<>();
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            partitions.forEach(partition -> 
topicToMemberSubscription.computeIfAbsent(partition.topicId(), k -> new 
LinkedHashSet<>()).add(member)));
+
+        Map<TopicIdPartition, List<String>> newAssignment = new HashMap<>();
+
+        // Step 1: Hash member IDs to partitions.
+        memberToPartitionsSubscription.forEach((member, partitions) ->
+            memberHashAssignment(partitions, List.of(member), newAssignment));
+
+        // Step 2: Round-robin assignment for unassigned partitions which do 
not have members already assigned in the current assignment.
+        Set<TopicIdPartition> assignedPartitions = new 
LinkedHashSet<>(newAssignment.keySet());
+        Map<Uuid, List<TopicIdPartition>> unassignedPartitions = new 
HashMap<>();
+        targetPartitions.forEach(targetPartition -> {
+            if (!assignedPartitions.contains(targetPartition) && 
!currentAssignment.containsKey(targetPartition))
+                
unassignedPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
ArrayList<>()).add(targetPartition);
+        });
+
+        unassignedPartitions.keySet().forEach(unassignedTopic ->
+            
roundRobinAssignment(topicToMemberSubscription.get(unassignedTopic), 
unassignedPartitions.get(unassignedTopic), newAssignment));
+
+        // Step 3: We combine current assignment and new assignment.
+        Map<String, Set<TopicIdPartition>> finalAssignment = new HashMap<>();
+
+        // When combining current assignment, we need to only consider the 
member topic subscription in current assignment
+        // which is being subscribed in the new assignment as well.
+        currentAssignment.forEach((targetPartition, members) -> 
members.forEach(member -> {
+            if 
(topicToMemberSubscription.getOrDefault(targetPartition.topicId(), 
Collections.emptySet()).contains(member))
+                finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition);
+        }));
+        newAssignment.forEach((targetPartition, members) -> 
members.forEach(member ->
+            finalAssignment.computeIfAbsent(member, k -> new 
HashSet<>()).add(targetPartition)));
+
+        return groupAssignment(finalAssignment, groupSpec.memberIds());
+    }
+
+    private GroupAssignment groupAssignment(
+        Map<String, Set<TopicIdPartition>> assignmentByMember,
+        Collection<String> allGroupMembers
+    ) {
+        Map<String, MemberAssignment> members = new HashMap<>();
+        for (Map.Entry<String, Set<TopicIdPartition>> entry : 
assignmentByMember.entrySet()) {
+            Map<Uuid, Set<Integer>> targetPartitions = new HashMap<>();
+            entry.getValue().forEach(targetPartition -> 
targetPartitions.computeIfAbsent(targetPartition.topicId(), k -> new 
HashSet<>()).add(targetPartition.partitionId()));
+            members.put(entry.getKey(), new 
MemberAssignmentImpl(targetPartitions));
+        }
+        allGroupMembers.forEach(member -> {
+            if (!members.containsKey(member))
+                members.put(member, new MemberAssignmentImpl(new HashMap<>()));
+        });
+
         return new GroupAssignment(members);
     }
 
-    private Map<Uuid, Set<Integer>> computeTargetPartitions(
-        Set<Uuid> subscribeTopicIds,
+    /**
+     * This function updates assignment by hashing the member IDs of the 
members and maps the partitions assigned to the members based on the hash. This 
gives approximately even balance.
+     * @param targetPartitions - the subscribed topic partitions.
+     * @param memberIds - the member ids to which the topic partitions need to 
be assigned.
+     * @param assignment - the existing assignment by topic partition. We need 
to pass it as a parameter because this
+     *                   function would be called multiple times for 
heterogeneous assignment.
+     */
+    void memberHashAssignment(

Review Comment:
   ```suggestion
        *                   function would be called multiple times for 
heterogeneous assignment.
        */
        // Visible for testing
       void memberHashAssignment(
   ```



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