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


##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec;
+import org.apache.kafka.coordinator.group.assignor.AssignmentSpec;
+import org.apache.kafka.coordinator.group.assignor.GroupAssignment;
+import org.apache.kafka.coordinator.group.assignor.MemberAssignment;
+import org.apache.kafka.coordinator.group.assignor.PartitionAssignor;
+import org.apache.kafka.coordinator.group.assignor.RangeAssignor;
+import org.apache.kafka.coordinator.group.assignor.SubscribedTopicDescriber;
+import org.apache.kafka.coordinator.group.assignor.UniformAssignor;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import static java.lang.Integer.max;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 1)
+@Measurement(iterations = 0)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class ServerSideAssignorBenchmark {
+
+    public enum AssignorType {
+        RANGE(new RangeAssignor()),
+        UNIFORM(new UniformAssignor());
+
+        private final PartitionAssignor assignor;
+
+        AssignorType(PartitionAssignor assignor) {
+            this.assignor = assignor;
+        }
+
+        public PartitionAssignor assignor() {
+            return assignor;
+        }
+    }
+
+    /**
+     * The subscription pattern followed by the members of the group.
+     *
+     * A subscription model is considered homogenous if all the members of the 
group
+     * are subscribed to the same set of topics, it is heterogeneous otherwise.
+     */
+    public enum SubscriptionModel {
+        HOMOGENEOUS, HETEROGENEOUS
+    }
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;
+
+    @Param({"HOMOGENEOUS", "HETEROGENEOUS"})
+    private SubscriptionModel subscriptionModel;
+
+    @Param({"RANGE", "UNIFORM"})
+    private AssignorType assignorType;
+
+    @Param({"true", "false"})
+    private boolean simulateRebalanceTrigger;
+
+    private PartitionAssignor partitionAssignor;
+
+    private static final int NUMBER_OF_RACKS = 3;
+
+    private AssignmentSpec assignmentSpec;
+
+    private SubscribedTopicDescriber subscribedTopicDescriber;
+
+    private final List<Uuid> allTopicIds = new ArrayList<>(topicCount);
+
+    @Setup(Level.Trial)
+    public void setup() {
+        Map<Uuid, TopicMetadata> topicMetadata = createTopicMetadata();
+        subscribedTopicDescriber = new SubscribedTopicMetadata(topicMetadata);
+
+        createAssignmentSpec();
+
+        partitionAssignor = assignorType.assignor();
+
+        if (simulateRebalanceTrigger) {
+            simulateIncrementalRebalance(topicMetadata);
+        }
+    }
+
+    private Map<Uuid, TopicMetadata> createTopicMetadata() {
+        Map<Uuid, TopicMetadata> topicMetadata = new HashMap<>();
+        int partitionsPerTopicCount = (memberCount * partitionsToMemberRatio) 
/ topicCount;
+
+        Map<Integer, Set<String>> partitionRacks = isRackAware ?
+            mkMapOfPartitionRacks(partitionsPerTopicCount) :
+            Collections.emptyMap();
+
+        for (int i = 0; i < topicCount; i++) {
+            Uuid topicUuid = Uuid.randomUuid();
+            String topicName = "topic" + i;
+            allTopicIds.add(topicUuid);
+            topicMetadata.put(topicUuid, new TopicMetadata(
+                topicUuid,
+                topicName,
+                partitionsPerTopicCount,
+                partitionRacks
+            ));
+        }
+
+        return topicMetadata;
+    }
+
+    private void createAssignmentSpec() {
+        Map<String, AssignmentMemberSpec> members = new HashMap<>();
+
+        int topicCounter = 0;
+        Map<Integer, Set<Uuid>> tempMemberIndexToSubscriptions = new 
HashMap<>(memberCount);
+
+        // In the rebalance case, we will add the last member as a trigger.
+        // This is done to keep the total members count consistent with the 
input.
+        int numberOfMembers = simulateRebalanceTrigger ? memberCount -1 : 
memberCount;
+
+        for (int i = 0; i < numberOfMembers; i++) {
+            if (subscriptionModel == SubscriptionModel.HOMOGENEOUS) {
+                addMemberSpec(members, i, new HashSet<>(allTopicIds));
+            } else {
+                Set<Uuid> subscribedTopics = new HashSet<>(Arrays.asList(
+                    allTopicIds.get(i % topicCount),
+                    allTopicIds.get((i+1) % topicCount)
+                ));
+                topicCounter = max (topicCounter, ((i+1) % topicCount));
+                tempMemberIndexToSubscriptions.put(i, subscribedTopics);
+            }
+        }
+
+        int lastAssignedTopicIndex = topicCounter;
+        tempMemberIndexToSubscriptions.forEach((memberIndex, subscriptions) -> 
{
+            if (lastAssignedTopicIndex < topicCount - 1) {
+                
subscriptions.addAll(allTopicIds.subList(lastAssignedTopicIndex + 1, topicCount 
- 1));
+            }
+            addMemberSpec(members, memberIndex, subscriptions);
+        });
+
+        this.assignmentSpec = new AssignmentSpec(members);
+    }
+
+    private Optional<String> rackId(int index) {
+        return isRackAware ? Optional.of("rack" + index % NUMBER_OF_RACKS) : 
Optional.empty();
+    }
+
+    private void addMemberSpec(
+        Map<String, AssignmentMemberSpec> members,
+        int memberIndex,
+        Set<Uuid> subscribedTopicIds
+    ) {
+        String memberName = "member" + memberIndex;

Review Comment:
   nit: Let's use `memberId` to be consistent with the assignor.



##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec;
+import org.apache.kafka.coordinator.group.assignor.AssignmentSpec;
+import org.apache.kafka.coordinator.group.assignor.GroupAssignment;
+import org.apache.kafka.coordinator.group.assignor.MemberAssignment;
+import org.apache.kafka.coordinator.group.assignor.PartitionAssignor;
+import org.apache.kafka.coordinator.group.assignor.RangeAssignor;
+import org.apache.kafka.coordinator.group.assignor.SubscribedTopicDescriber;
+import org.apache.kafka.coordinator.group.assignor.UniformAssignor;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import static java.lang.Integer.max;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 1)
+@Measurement(iterations = 0)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class ServerSideAssignorBenchmark {
+
+    public enum AssignorType {
+        RANGE(new RangeAssignor()),
+        UNIFORM(new UniformAssignor());
+
+        private final PartitionAssignor assignor;
+
+        AssignorType(PartitionAssignor assignor) {
+            this.assignor = assignor;
+        }
+
+        public PartitionAssignor assignor() {
+            return assignor;
+        }
+    }
+
+    /**
+     * The subscription pattern followed by the members of the group.
+     *
+     * A subscription model is considered homogenous if all the members of the 
group
+     * are subscribed to the same set of topics, it is heterogeneous otherwise.
+     */
+    public enum SubscriptionModel {
+        HOMOGENEOUS, HETEROGENEOUS
+    }
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;
+
+    @Param({"HOMOGENEOUS", "HETEROGENEOUS"})
+    private SubscriptionModel subscriptionModel;
+
+    @Param({"RANGE", "UNIFORM"})
+    private AssignorType assignorType;
+
+    @Param({"true", "false"})
+    private boolean simulateRebalanceTrigger;

Review Comment:
   nit: I have a small preference for an enum with FULL and INCREMENTAL. What 
do you think?



##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec;
+import org.apache.kafka.coordinator.group.assignor.AssignmentSpec;
+import org.apache.kafka.coordinator.group.assignor.GroupAssignment;
+import org.apache.kafka.coordinator.group.assignor.MemberAssignment;
+import org.apache.kafka.coordinator.group.assignor.PartitionAssignor;
+import org.apache.kafka.coordinator.group.assignor.RangeAssignor;
+import org.apache.kafka.coordinator.group.assignor.SubscribedTopicDescriber;
+import org.apache.kafka.coordinator.group.assignor.UniformAssignor;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import static java.lang.Integer.max;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 1)
+@Measurement(iterations = 0)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class ServerSideAssignorBenchmark {
+
+    public enum AssignorType {
+        RANGE(new RangeAssignor()),
+        UNIFORM(new UniformAssignor());
+
+        private final PartitionAssignor assignor;
+
+        AssignorType(PartitionAssignor assignor) {
+            this.assignor = assignor;
+        }
+
+        public PartitionAssignor assignor() {
+            return assignor;
+        }
+    }
+
+    /**
+     * The subscription pattern followed by the members of the group.
+     *
+     * A subscription model is considered homogenous if all the members of the 
group
+     * are subscribed to the same set of topics, it is heterogeneous otherwise.
+     */
+    public enum SubscriptionModel {
+        HOMOGENEOUS, HETEROGENEOUS
+    }
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;
+
+    @Param({"HOMOGENEOUS", "HETEROGENEOUS"})
+    private SubscriptionModel subscriptionModel;
+
+    @Param({"RANGE", "UNIFORM"})
+    private AssignorType assignorType;
+
+    @Param({"true", "false"})
+    private boolean simulateRebalanceTrigger;
+
+    private PartitionAssignor partitionAssignor;
+
+    private static final int NUMBER_OF_RACKS = 3;
+
+    private AssignmentSpec assignmentSpec;
+
+    private SubscribedTopicDescriber subscribedTopicDescriber;
+
+    private final List<Uuid> allTopicIds = new ArrayList<>(topicCount);
+
+    @Setup(Level.Trial)
+    public void setup() {
+        Map<Uuid, TopicMetadata> topicMetadata = createTopicMetadata();
+        subscribedTopicDescriber = new SubscribedTopicMetadata(topicMetadata);
+
+        createAssignmentSpec();
+
+        partitionAssignor = assignorType.assignor();
+
+        if (simulateRebalanceTrigger) {
+            simulateIncrementalRebalance(topicMetadata);
+        }
+    }
+
+    private Map<Uuid, TopicMetadata> createTopicMetadata() {
+        Map<Uuid, TopicMetadata> topicMetadata = new HashMap<>();
+        int partitionsPerTopicCount = (memberCount * partitionsToMemberRatio) 
/ topicCount;
+
+        Map<Integer, Set<String>> partitionRacks = isRackAware ?
+            mkMapOfPartitionRacks(partitionsPerTopicCount) :
+            Collections.emptyMap();
+
+        for (int i = 0; i < topicCount; i++) {
+            Uuid topicUuid = Uuid.randomUuid();
+            String topicName = "topic" + i;
+            allTopicIds.add(topicUuid);
+            topicMetadata.put(topicUuid, new TopicMetadata(
+                topicUuid,
+                topicName,
+                partitionsPerTopicCount,
+                partitionRacks
+            ));
+        }
+
+        return topicMetadata;
+    }
+
+    private void createAssignmentSpec() {
+        Map<String, AssignmentMemberSpec> members = new HashMap<>();
+
+        int topicCounter = 0;
+        Map<Integer, Set<Uuid>> tempMemberIndexToSubscriptions = new 
HashMap<>(memberCount);
+
+        // In the rebalance case, we will add the last member as a trigger.
+        // This is done to keep the total members count consistent with the 
input.
+        int numberOfMembers = simulateRebalanceTrigger ? memberCount -1 : 
memberCount;
+
+        for (int i = 0; i < numberOfMembers; i++) {
+            if (subscriptionModel == SubscriptionModel.HOMOGENEOUS) {
+                addMemberSpec(members, i, new HashSet<>(allTopicIds));
+            } else {
+                Set<Uuid> subscribedTopics = new HashSet<>(Arrays.asList(
+                    allTopicIds.get(i % topicCount),
+                    allTopicIds.get((i+1) % topicCount)
+                ));
+                topicCounter = max (topicCounter, ((i+1) % topicCount));
+                tempMemberIndexToSubscriptions.put(i, subscribedTopics);
+            }
+        }
+
+        int lastAssignedTopicIndex = topicCounter;
+        tempMemberIndexToSubscriptions.forEach((memberIndex, subscriptions) -> 
{
+            if (lastAssignedTopicIndex < topicCount - 1) {
+                
subscriptions.addAll(allTopicIds.subList(lastAssignedTopicIndex + 1, topicCount 
- 1));
+            }
+            addMemberSpec(members, memberIndex, subscriptions);
+        });
+
+        this.assignmentSpec = new AssignmentSpec(members);
+    }
+
+    private Optional<String> rackId(int index) {
+        return isRackAware ? Optional.of("rack" + index % NUMBER_OF_RACKS) : 
Optional.empty();
+    }
+
+    private void addMemberSpec(
+        Map<String, AssignmentMemberSpec> members,
+        int memberIndex,
+        Set<Uuid> subscribedTopicIds
+    ) {
+        String memberName = "member" + memberIndex;
+        Optional<String> rackId = rackId(memberIndex);
+
+        members.put(memberName, new AssignmentMemberSpec(
+            Optional.empty(),
+            rackId,
+            subscribedTopicIds,
+            Collections.emptyMap()
+        ));
+    }
+
+    private static Map<Integer, Set<String>> mkMapOfPartitionRacks(int 
numPartitions) {
+        Map<Integer, Set<String>> partitionRacks = new 
HashMap<>(numPartitions);
+        for (int i = 0; i < numPartitions; i++) {
+            partitionRacks.put(i, new HashSet<>(Arrays.asList("rack" + i % 
NUMBER_OF_RACKS, "rack" + (i + 1) % NUMBER_OF_RACKS)));

Review Comment:
   In practice, partitions will have three racks. Should we have three here?



##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec;
+import org.apache.kafka.coordinator.group.assignor.AssignmentSpec;
+import org.apache.kafka.coordinator.group.assignor.GroupAssignment;
+import org.apache.kafka.coordinator.group.assignor.MemberAssignment;
+import org.apache.kafka.coordinator.group.assignor.PartitionAssignor;
+import org.apache.kafka.coordinator.group.assignor.UniformAssignor;
+import org.apache.kafka.coordinator.group.consumer.Assignment;
+import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.VersionedMetadata;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 5)
+@Measurement(iterations = 5)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class TargetAssignmentBuilderBenchmark {
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;
+
+    String groupId = "benchmark-group";
+
+    private static final int groupEpoch = 0;

Review Comment:
   nit: Let's format it as a constant.



##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec;
+import org.apache.kafka.coordinator.group.assignor.AssignmentSpec;
+import org.apache.kafka.coordinator.group.assignor.GroupAssignment;
+import org.apache.kafka.coordinator.group.assignor.MemberAssignment;
+import org.apache.kafka.coordinator.group.assignor.PartitionAssignor;
+import org.apache.kafka.coordinator.group.assignor.UniformAssignor;
+import org.apache.kafka.coordinator.group.consumer.Assignment;
+import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.VersionedMetadata;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 5)
+@Measurement(iterations = 5)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class TargetAssignmentBuilderBenchmark {
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;
+
+    String groupId = "benchmark-group";
+
+    private static final int groupEpoch = 0;
+
+    private PartitionAssignor partitionAssignor;
+
+    private Map<String, TopicMetadata> subscriptionMetadata = 
Collections.emptyMap();
+
+    private TargetAssignmentBuilder targetAssignmentBuilder;
+
+    private AssignmentSpec assignmentSpec;
+
+    private static final int numberOfRacks = 3;
+
+    private final List<String> allTopicNames = new ArrayList<>(topicCount);
+
+    private final List<Uuid> allTopicIds = new ArrayList<>(topicCount);
+
+    @Setup(Level.Trial)
+    public void setup() {
+        // For this benchmark we will use the Uniform Assignor
+        // and a group that has a homogeneous subscription model.
+        partitionAssignor = new UniformAssignor();
+        subscriptionMetadata = generateMockSubscriptionMetadata();
+        Map<String, ConsumerGroupMember> members = generateMockMembers();
+        Map<String, Assignment> existingTargetAssignment = 
generateMockInitialTargetAssignment();
+
+        // Add a new member to trigger a rebalance.
+        Set<String> subscribedTopics = new 
HashSet<>(subscriptionMetadata.keySet());
+        String rackId = isRackAware ? "rack" + (memberCount - 1) % 
numberOfRacks : "";

Review Comment:
   nit: Should we extract it into an helper method too?



##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec;
+import org.apache.kafka.coordinator.group.assignor.AssignmentSpec;
+import org.apache.kafka.coordinator.group.assignor.GroupAssignment;
+import org.apache.kafka.coordinator.group.assignor.MemberAssignment;
+import org.apache.kafka.coordinator.group.assignor.PartitionAssignor;
+import org.apache.kafka.coordinator.group.assignor.UniformAssignor;
+import org.apache.kafka.coordinator.group.consumer.Assignment;
+import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.VersionedMetadata;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 5)
+@Measurement(iterations = 5)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class TargetAssignmentBuilderBenchmark {
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;
+
+    String groupId = "benchmark-group";
+
+    private static final int groupEpoch = 0;
+
+    private PartitionAssignor partitionAssignor;
+
+    private Map<String, TopicMetadata> subscriptionMetadata = 
Collections.emptyMap();
+
+    private TargetAssignmentBuilder targetAssignmentBuilder;
+
+    private AssignmentSpec assignmentSpec;
+
+    private static final int numberOfRacks = 3;
+
+    private final List<String> allTopicNames = new ArrayList<>(topicCount);
+
+    private final List<Uuid> allTopicIds = new ArrayList<>(topicCount);
+
+    @Setup(Level.Trial)
+    public void setup() {
+        // For this benchmark we will use the Uniform Assignor
+        // and a group that has a homogeneous subscription model.
+        partitionAssignor = new UniformAssignor();
+        subscriptionMetadata = generateMockSubscriptionMetadata();
+        Map<String, ConsumerGroupMember> members = generateMockMembers();
+        Map<String, Assignment> existingTargetAssignment = 
generateMockInitialTargetAssignment();
+
+        // Add a new member to trigger a rebalance.
+        Set<String> subscribedTopics = new 
HashSet<>(subscriptionMetadata.keySet());
+        String rackId = isRackAware ? "rack" + (memberCount - 1) % 
numberOfRacks : "";
+        ConsumerGroupMember newMember = new 
ConsumerGroupMember.Builder("new-member")
+            .setSubscribedTopicNames(new ArrayList<>(subscribedTopics))
+            .setRackId(rackId)
+            .build();
+
+        targetAssignmentBuilder = new TargetAssignmentBuilder(groupId, 
groupEpoch, partitionAssignor)
+            .withMembers(members)
+            .withSubscriptionMetadata(subscriptionMetadata)
+            .withTargetAssignment(existingTargetAssignment)
+            .addOrUpdateMember(newMember.memberId(), newMember);
+    }
+
+    private Map<String, ConsumerGroupMember> generateMockMembers() {
+        Map<String, ConsumerGroupMember> members = new HashMap<>();
+
+        for (int i = 0; i < memberCount - 1; i++) {
+            Set<String> subscribedTopics;
+            subscribedTopics = new HashSet<>(allTopicNames);
+
+            String rackId = isRackAware ? "rack" + i % numberOfRacks : "" ;
+            ConsumerGroupMember member = new 
ConsumerGroupMember.Builder("member" + i)
+                .setSubscribedTopicNames(new ArrayList<>(subscribedTopics))
+                .setRackId(rackId)
+                .build();
+            members.put("member" + i, member);
+        }
+        return members;
+    }
+
+    private Map<String, TopicMetadata> generateMockSubscriptionMetadata() {
+        Map<String, TopicMetadata> subscriptionMetadata = new HashMap<>();
+        int partitionsPerTopicCount = (memberCount * partitionsToMemberRatio) 
/ topicCount;
+
+        for (int i = 0; i < topicCount; i++) {
+            String topicName = "topic-" + i;
+            Uuid topicId = Uuid.randomUuid();
+            allTopicNames.add(topicName);
+            allTopicIds.add(topicId);
+            Map<Integer, Set<String>> partitionRacks = 
mkMapOfPartitionRacks(partitionsPerTopicCount);
+            TopicMetadata metadata = new TopicMetadata(topicId, topicName, 
partitionsPerTopicCount, partitionRacks);
+            subscriptionMetadata.put(topicName, metadata);
+        }
+
+        return subscriptionMetadata;
+    }
+
+    private Map<String, Assignment> generateMockInitialTargetAssignment() {
+        Map<Uuid, TopicMetadata> topicMetadataMap = new HashMap<>(topicCount);
+        subscriptionMetadata.forEach((topicName, topicMetadata) ->
+            topicMetadataMap.put(
+                topicMetadata.id(),
+                topicMetadata
+            )
+        );
+
+        createAssignmentSpec();
+
+        GroupAssignment groupAssignment = partitionAssignor.assign(
+            assignmentSpec,
+            new SubscribedTopicMetadata(topicMetadataMap)
+        );
+
+        Map<String, Assignment> initialTargetAssignment = new 
HashMap<>(memberCount);
+
+        for (Map.Entry<String, MemberAssignment> entry : 
groupAssignment.members().entrySet()) {
+            String memberId = entry.getKey();
+            Map<Uuid, Set<Integer>> topicPartitions = 
entry.getValue().targetPartitions();
+
+            Assignment assignment = new Assignment((byte) 0, topicPartitions, 
VersionedMetadata.EMPTY);
+
+            initialTargetAssignment.put(memberId, assignment);
+        }
+
+        return initialTargetAssignment;
+    }
+
+    private static Map<Integer, Set<String>> mkMapOfPartitionRacks(int 
numPartitions) {
+        Map<Integer, Set<String>> partitionRacks = new 
HashMap<>(numPartitions);
+        for (int i = 0; i < numPartitions; i++) {
+            partitionRacks.put(i, new HashSet<>(Arrays.asList("rack" + i % 
numberOfRacks, "rack" + (i + 1) % numberOfRacks)));
+        }
+        return partitionRacks;
+    }
+
+    private Optional<String> rackId(int index) {
+        return isRackAware ? Optional.of("rack" + index % numberOfRacks) : 
Optional.empty();
+    }
+
+    private void createAssignmentSpec() {
+        Map<String, AssignmentMemberSpec> members = new HashMap<>();
+
+        for (int i = 0; i < memberCount - 1; i++) {
+            String memberName = "member" + i;
+            Optional<String> rackId = rackId(i);
+
+            members.put(memberName, new AssignmentMemberSpec(
+                Optional.empty(),
+                rackId,
+                allTopicIds,
+                Collections.emptyMap()
+            ));
+        }
+        this.assignmentSpec = new AssignmentSpec(members);
+    }
+
+    @Benchmark
+    @Threads(1)
+    @OutputTimeUnit(TimeUnit.MILLISECONDS)
+    public void build() {
+        this.targetAssignmentBuilder.build();

Review Comment:
   ditto.



##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec;
+import org.apache.kafka.coordinator.group.assignor.AssignmentSpec;
+import org.apache.kafka.coordinator.group.assignor.GroupAssignment;
+import org.apache.kafka.coordinator.group.assignor.MemberAssignment;
+import org.apache.kafka.coordinator.group.assignor.PartitionAssignor;
+import org.apache.kafka.coordinator.group.assignor.UniformAssignor;
+import org.apache.kafka.coordinator.group.consumer.Assignment;
+import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.VersionedMetadata;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 5)
+@Measurement(iterations = 5)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class TargetAssignmentBuilderBenchmark {
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;
+
+    String groupId = "benchmark-group";
+
+    private static final int groupEpoch = 0;
+
+    private PartitionAssignor partitionAssignor;
+
+    private Map<String, TopicMetadata> subscriptionMetadata = 
Collections.emptyMap();
+
+    private TargetAssignmentBuilder targetAssignmentBuilder;
+
+    private AssignmentSpec assignmentSpec;
+
+    private static final int numberOfRacks = 3;

Review Comment:
   ditto.



##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec;
+import org.apache.kafka.coordinator.group.assignor.AssignmentSpec;
+import org.apache.kafka.coordinator.group.assignor.GroupAssignment;
+import org.apache.kafka.coordinator.group.assignor.MemberAssignment;
+import org.apache.kafka.coordinator.group.assignor.PartitionAssignor;
+import org.apache.kafka.coordinator.group.assignor.UniformAssignor;
+import org.apache.kafka.coordinator.group.consumer.Assignment;
+import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.VersionedMetadata;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 5)
+@Measurement(iterations = 5)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class TargetAssignmentBuilderBenchmark {
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;
+
+    String groupId = "benchmark-group";
+
+    private static final int groupEpoch = 0;
+
+    private PartitionAssignor partitionAssignor;
+
+    private Map<String, TopicMetadata> subscriptionMetadata = 
Collections.emptyMap();
+
+    private TargetAssignmentBuilder targetAssignmentBuilder;
+
+    private AssignmentSpec assignmentSpec;
+
+    private static final int numberOfRacks = 3;
+
+    private final List<String> allTopicNames = new ArrayList<>(topicCount);
+
+    private final List<Uuid> allTopicIds = new ArrayList<>(topicCount);
+
+    @Setup(Level.Trial)
+    public void setup() {
+        // For this benchmark we will use the Uniform Assignor
+        // and a group that has a homogeneous subscription model.
+        partitionAssignor = new UniformAssignor();
+        subscriptionMetadata = generateMockSubscriptionMetadata();
+        Map<String, ConsumerGroupMember> members = generateMockMembers();
+        Map<String, Assignment> existingTargetAssignment = 
generateMockInitialTargetAssignment();
+
+        // Add a new member to trigger a rebalance.
+        Set<String> subscribedTopics = new 
HashSet<>(subscriptionMetadata.keySet());
+        String rackId = isRackAware ? "rack" + (memberCount - 1) % 
numberOfRacks : "";

Review Comment:
   It is already defined. Let's use it.



##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec;
+import org.apache.kafka.coordinator.group.assignor.AssignmentSpec;
+import org.apache.kafka.coordinator.group.assignor.GroupAssignment;
+import org.apache.kafka.coordinator.group.assignor.MemberAssignment;
+import org.apache.kafka.coordinator.group.assignor.PartitionAssignor;
+import org.apache.kafka.coordinator.group.assignor.UniformAssignor;
+import org.apache.kafka.coordinator.group.consumer.Assignment;
+import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.VersionedMetadata;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 5)
+@Measurement(iterations = 5)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class TargetAssignmentBuilderBenchmark {
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;
+
+    String groupId = "benchmark-group";

Review Comment:
   This one seems to be a constant. Let's format it appropriately and make it 
static final.



##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec;
+import org.apache.kafka.coordinator.group.assignor.AssignmentSpec;
+import org.apache.kafka.coordinator.group.assignor.GroupAssignment;
+import org.apache.kafka.coordinator.group.assignor.MemberAssignment;
+import org.apache.kafka.coordinator.group.assignor.PartitionAssignor;
+import org.apache.kafka.coordinator.group.assignor.RangeAssignor;
+import org.apache.kafka.coordinator.group.assignor.SubscribedTopicDescriber;
+import org.apache.kafka.coordinator.group.assignor.UniformAssignor;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import static java.lang.Integer.max;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 1)
+@Measurement(iterations = 0)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class ServerSideAssignorBenchmark {
+
+    public enum AssignorType {
+        RANGE(new RangeAssignor()),
+        UNIFORM(new UniformAssignor());
+
+        private final PartitionAssignor assignor;
+
+        AssignorType(PartitionAssignor assignor) {
+            this.assignor = assignor;
+        }
+
+        public PartitionAssignor assignor() {
+            return assignor;
+        }
+    }
+
+    /**
+     * The subscription pattern followed by the members of the group.
+     *
+     * A subscription model is considered homogenous if all the members of the 
group
+     * are subscribed to the same set of topics, it is heterogeneous otherwise.
+     */
+    public enum SubscriptionModel {
+        HOMOGENEOUS, HETEROGENEOUS
+    }
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;
+
+    @Param({"HOMOGENEOUS", "HETEROGENEOUS"})
+    private SubscriptionModel subscriptionModel;
+
+    @Param({"RANGE", "UNIFORM"})
+    private AssignorType assignorType;
+
+    @Param({"true", "false"})
+    private boolean simulateRebalanceTrigger;
+
+    private PartitionAssignor partitionAssignor;
+
+    private static final int NUMBER_OF_RACKS = 3;
+
+    private AssignmentSpec assignmentSpec;
+
+    private SubscribedTopicDescriber subscribedTopicDescriber;
+
+    private final List<Uuid> allTopicIds = new ArrayList<>(topicCount);
+
+    @Setup(Level.Trial)
+    public void setup() {
+        Map<Uuid, TopicMetadata> topicMetadata = createTopicMetadata();
+        subscribedTopicDescriber = new SubscribedTopicMetadata(topicMetadata);
+
+        createAssignmentSpec();
+
+        partitionAssignor = assignorType.assignor();
+
+        if (simulateRebalanceTrigger) {
+            simulateIncrementalRebalance(topicMetadata);
+        }
+    }
+
+    private Map<Uuid, TopicMetadata> createTopicMetadata() {
+        Map<Uuid, TopicMetadata> topicMetadata = new HashMap<>();
+        int partitionsPerTopicCount = (memberCount * partitionsToMemberRatio) 
/ topicCount;
+
+        Map<Integer, Set<String>> partitionRacks = isRackAware ?
+            mkMapOfPartitionRacks(partitionsPerTopicCount) :
+            Collections.emptyMap();
+
+        for (int i = 0; i < topicCount; i++) {
+            Uuid topicUuid = Uuid.randomUuid();
+            String topicName = "topic" + i;
+            allTopicIds.add(topicUuid);
+            topicMetadata.put(topicUuid, new TopicMetadata(
+                topicUuid,
+                topicName,
+                partitionsPerTopicCount,
+                partitionRacks
+            ));
+        }
+
+        return topicMetadata;
+    }
+
+    private void createAssignmentSpec() {
+        Map<String, AssignmentMemberSpec> members = new HashMap<>();
+
+        int topicCounter = 0;
+        Map<Integer, Set<Uuid>> tempMemberIndexToSubscriptions = new 
HashMap<>(memberCount);
+
+        // In the rebalance case, we will add the last member as a trigger.
+        // This is done to keep the total members count consistent with the 
input.
+        int numberOfMembers = simulateRebalanceTrigger ? memberCount -1 : 
memberCount;
+
+        for (int i = 0; i < numberOfMembers; i++) {
+            if (subscriptionModel == SubscriptionModel.HOMOGENEOUS) {
+                addMemberSpec(members, i, new HashSet<>(allTopicIds));
+            } else {
+                Set<Uuid> subscribedTopics = new HashSet<>(Arrays.asList(
+                    allTopicIds.get(i % topicCount),
+                    allTopicIds.get((i+1) % topicCount)
+                ));
+                topicCounter = max (topicCounter, ((i+1) % topicCount));
+                tempMemberIndexToSubscriptions.put(i, subscribedTopics);
+            }
+        }
+
+        int lastAssignedTopicIndex = topicCounter;
+        tempMemberIndexToSubscriptions.forEach((memberIndex, subscriptions) -> 
{
+            if (lastAssignedTopicIndex < topicCount - 1) {
+                
subscriptions.addAll(allTopicIds.subList(lastAssignedTopicIndex + 1, topicCount 
- 1));
+            }
+            addMemberSpec(members, memberIndex, subscriptions);
+        });
+
+        this.assignmentSpec = new AssignmentSpec(members);
+    }
+
+    private Optional<String> rackId(int index) {
+        return isRackAware ? Optional.of("rack" + index % NUMBER_OF_RACKS) : 
Optional.empty();
+    }
+
+    private void addMemberSpec(
+        Map<String, AssignmentMemberSpec> members,
+        int memberIndex,
+        Set<Uuid> subscribedTopicIds
+    ) {
+        String memberName = "member" + memberIndex;
+        Optional<String> rackId = rackId(memberIndex);
+
+        members.put(memberName, new AssignmentMemberSpec(
+            Optional.empty(),
+            rackId,
+            subscribedTopicIds,
+            Collections.emptyMap()
+        ));
+    }
+
+    private static Map<Integer, Set<String>> mkMapOfPartitionRacks(int 
numPartitions) {
+        Map<Integer, Set<String>> partitionRacks = new 
HashMap<>(numPartitions);
+        for (int i = 0; i < numPartitions; i++) {
+            partitionRacks.put(i, new HashSet<>(Arrays.asList("rack" + i % 
NUMBER_OF_RACKS, "rack" + (i + 1) % NUMBER_OF_RACKS)));
+        }
+        return partitionRacks;
+    }
+
+    private void simulateIncrementalRebalance(Map<Uuid, TopicMetadata> 
topicMetadata) {
+        GroupAssignment initialAssignment = 
partitionAssignor.assign(assignmentSpec, subscribedTopicDescriber);
+        Map<String, MemberAssignment> members = initialAssignment.members();
+
+        Map<String, AssignmentMemberSpec> updatedMembers = new HashMap<>();
+        members.forEach((memberId, memberAssignment) -> {
+            AssignmentMemberSpec memberSpec = 
assignmentSpec.members().get(memberId);
+            updatedMembers.put(memberId, new AssignmentMemberSpec(
+                memberSpec.instanceId(),
+                memberSpec.rackId(),
+                memberSpec.subscribedTopicIds(),
+                memberAssignment.targetPartitions()
+            ));
+        });
+
+        Optional<String> rackId = rackId(memberCount-1);

Review Comment:
   nit: Spaces are missing around the `-` sign.



##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec;
+import org.apache.kafka.coordinator.group.assignor.AssignmentSpec;
+import org.apache.kafka.coordinator.group.assignor.GroupAssignment;
+import org.apache.kafka.coordinator.group.assignor.MemberAssignment;
+import org.apache.kafka.coordinator.group.assignor.PartitionAssignor;
+import org.apache.kafka.coordinator.group.assignor.UniformAssignor;
+import org.apache.kafka.coordinator.group.consumer.Assignment;
+import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.VersionedMetadata;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 5)
+@Measurement(iterations = 5)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class TargetAssignmentBuilderBenchmark {
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;
+
+    String groupId = "benchmark-group";
+
+    private static final int groupEpoch = 0;
+
+    private PartitionAssignor partitionAssignor;
+
+    private Map<String, TopicMetadata> subscriptionMetadata = 
Collections.emptyMap();
+
+    private TargetAssignmentBuilder targetAssignmentBuilder;
+
+    private AssignmentSpec assignmentSpec;
+
+    private static final int numberOfRacks = 3;
+
+    private final List<String> allTopicNames = new ArrayList<>(topicCount);
+
+    private final List<Uuid> allTopicIds = new ArrayList<>(topicCount);
+
+    @Setup(Level.Trial)
+    public void setup() {
+        // For this benchmark we will use the Uniform Assignor
+        // and a group that has a homogeneous subscription model.
+        partitionAssignor = new UniformAssignor();
+        subscriptionMetadata = generateMockSubscriptionMetadata();
+        Map<String, ConsumerGroupMember> members = generateMockMembers();
+        Map<String, Assignment> existingTargetAssignment = 
generateMockInitialTargetAssignment();
+
+        // Add a new member to trigger a rebalance.
+        Set<String> subscribedTopics = new 
HashSet<>(subscriptionMetadata.keySet());
+        String rackId = isRackAware ? "rack" + (memberCount - 1) % 
numberOfRacks : "";
+        ConsumerGroupMember newMember = new 
ConsumerGroupMember.Builder("new-member")
+            .setSubscribedTopicNames(new ArrayList<>(subscribedTopics))
+            .setRackId(rackId)
+            .build();
+
+        targetAssignmentBuilder = new TargetAssignmentBuilder(groupId, 
groupEpoch, partitionAssignor)
+            .withMembers(members)
+            .withSubscriptionMetadata(subscriptionMetadata)
+            .withTargetAssignment(existingTargetAssignment)
+            .addOrUpdateMember(newMember.memberId(), newMember);
+    }
+
+    private Map<String, ConsumerGroupMember> generateMockMembers() {
+        Map<String, ConsumerGroupMember> members = new HashMap<>();
+
+        for (int i = 0; i < memberCount - 1; i++) {
+            Set<String> subscribedTopics;
+            subscribedTopics = new HashSet<>(allTopicNames);
+
+            String rackId = isRackAware ? "rack" + i % numberOfRacks : "" ;
+            ConsumerGroupMember member = new 
ConsumerGroupMember.Builder("member" + i)
+                .setSubscribedTopicNames(new ArrayList<>(subscribedTopics))
+                .setRackId(rackId)
+                .build();
+            members.put("member" + i, member);
+        }
+        return members;
+    }
+
+    private Map<String, TopicMetadata> generateMockSubscriptionMetadata() {
+        Map<String, TopicMetadata> subscriptionMetadata = new HashMap<>();
+        int partitionsPerTopicCount = (memberCount * partitionsToMemberRatio) 
/ topicCount;
+
+        for (int i = 0; i < topicCount; i++) {
+            String topicName = "topic-" + i;
+            Uuid topicId = Uuid.randomUuid();
+            allTopicNames.add(topicName);
+            allTopicIds.add(topicId);
+            Map<Integer, Set<String>> partitionRacks = 
mkMapOfPartitionRacks(partitionsPerTopicCount);
+            TopicMetadata metadata = new TopicMetadata(topicId, topicName, 
partitionsPerTopicCount, partitionRacks);
+            subscriptionMetadata.put(topicName, metadata);
+        }
+
+        return subscriptionMetadata;
+    }
+
+    private Map<String, Assignment> generateMockInitialTargetAssignment() {
+        Map<Uuid, TopicMetadata> topicMetadataMap = new HashMap<>(topicCount);
+        subscriptionMetadata.forEach((topicName, topicMetadata) ->
+            topicMetadataMap.put(
+                topicMetadata.id(),
+                topicMetadata
+            )
+        );
+
+        createAssignmentSpec();
+
+        GroupAssignment groupAssignment = partitionAssignor.assign(
+            assignmentSpec,
+            new SubscribedTopicMetadata(topicMetadataMap)
+        );
+
+        Map<String, Assignment> initialTargetAssignment = new 
HashMap<>(memberCount);
+
+        for (Map.Entry<String, MemberAssignment> entry : 
groupAssignment.members().entrySet()) {
+            String memberId = entry.getKey();
+            Map<Uuid, Set<Integer>> topicPartitions = 
entry.getValue().targetPartitions();
+
+            Assignment assignment = new Assignment((byte) 0, topicPartitions, 
VersionedMetadata.EMPTY);
+
+            initialTargetAssignment.put(memberId, assignment);
+        }
+
+        return initialTargetAssignment;
+    }
+
+    private static Map<Integer, Set<String>> mkMapOfPartitionRacks(int 
numPartitions) {
+        Map<Integer, Set<String>> partitionRacks = new 
HashMap<>(numPartitions);
+        for (int i = 0; i < numPartitions; i++) {
+            partitionRacks.put(i, new HashSet<>(Arrays.asList("rack" + i % 
numberOfRacks, "rack" + (i + 1) % numberOfRacks)));
+        }
+        return partitionRacks;
+    }
+
+    private Optional<String> rackId(int index) {
+        return isRackAware ? Optional.of("rack" + index % numberOfRacks) : 
Optional.empty();
+    }
+
+    private void createAssignmentSpec() {
+        Map<String, AssignmentMemberSpec> members = new HashMap<>();
+
+        for (int i = 0; i < memberCount - 1; i++) {
+            String memberName = "member" + i;
+            Optional<String> rackId = rackId(i);
+
+            members.put(memberName, new AssignmentMemberSpec(
+                Optional.empty(),
+                rackId,
+                allTopicIds,
+                Collections.emptyMap()
+            ));
+        }
+        this.assignmentSpec = new AssignmentSpec(members);

Review Comment:
   nit: We can remove `this.`.



##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java:
##########
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec;
+import org.apache.kafka.coordinator.group.assignor.AssignmentSpec;
+import org.apache.kafka.coordinator.group.assignor.GroupAssignment;
+import org.apache.kafka.coordinator.group.assignor.MemberAssignment;
+import org.apache.kafka.coordinator.group.assignor.PartitionAssignor;
+import org.apache.kafka.coordinator.group.assignor.RangeAssignor;
+import org.apache.kafka.coordinator.group.assignor.SubscribedTopicDescriber;
+import org.apache.kafka.coordinator.group.assignor.UniformAssignor;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+import static java.lang.Integer.max;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 1)
+@Measurement(iterations = 0)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class ServerSideAssignorBenchmark {
+
+    public enum AssignorType {
+        RANGE(new RangeAssignor()),
+        UNIFORM(new UniformAssignor());
+
+        private final PartitionAssignor assignor;
+
+        AssignorType(PartitionAssignor assignor) {
+            this.assignor = assignor;
+        }
+
+        public PartitionAssignor assignor() {
+            return assignor;
+        }
+    }
+
+    /**
+     * The subscription pattern followed by the members of the group.
+     *
+     * A subscription model is considered homogenous if all the members of the 
group
+     * are subscribed to the same set of topics, it is heterogeneous otherwise.
+     */
+    public enum SubscriptionModel {
+        HOMOGENEOUS, HETEROGENEOUS
+    }
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;
+
+    @Param({"HOMOGENEOUS", "HETEROGENEOUS"})
+    private SubscriptionModel subscriptionModel;
+
+    @Param({"RANGE", "UNIFORM"})
+    private AssignorType assignorType;
+
+    @Param({"true", "false"})
+    private boolean simulateRebalanceTrigger;
+
+    private PartitionAssignor partitionAssignor;
+
+    private static final int NUMBER_OF_RACKS = 3;
+
+    private AssignmentSpec assignmentSpec;
+
+    private SubscribedTopicDescriber subscribedTopicDescriber;
+
+    private final List<Uuid> allTopicIds = new ArrayList<>(topicCount);
+
+    @Setup(Level.Trial)
+    public void setup() {
+        Map<Uuid, TopicMetadata> topicMetadata = createTopicMetadata();
+        subscribedTopicDescriber = new SubscribedTopicMetadata(topicMetadata);
+
+        createAssignmentSpec();
+
+        partitionAssignor = assignorType.assignor();
+
+        if (simulateRebalanceTrigger) {
+            simulateIncrementalRebalance(topicMetadata);
+        }
+    }
+
+    private Map<Uuid, TopicMetadata> createTopicMetadata() {
+        Map<Uuid, TopicMetadata> topicMetadata = new HashMap<>();
+        int partitionsPerTopicCount = (memberCount * partitionsToMemberRatio) 
/ topicCount;
+
+        Map<Integer, Set<String>> partitionRacks = isRackAware ?
+            mkMapOfPartitionRacks(partitionsPerTopicCount) :
+            Collections.emptyMap();
+
+        for (int i = 0; i < topicCount; i++) {
+            Uuid topicUuid = Uuid.randomUuid();
+            String topicName = "topic" + i;
+            allTopicIds.add(topicUuid);
+            topicMetadata.put(topicUuid, new TopicMetadata(
+                topicUuid,
+                topicName,
+                partitionsPerTopicCount,
+                partitionRacks
+            ));
+        }
+
+        return topicMetadata;
+    }
+
+    private void createAssignmentSpec() {
+        Map<String, AssignmentMemberSpec> members = new HashMap<>();
+
+        int topicCounter = 0;
+        Map<Integer, Set<Uuid>> tempMemberIndexToSubscriptions = new 
HashMap<>(memberCount);
+
+        // In the rebalance case, we will add the last member as a trigger.
+        // This is done to keep the total members count consistent with the 
input.
+        int numberOfMembers = simulateRebalanceTrigger ? memberCount -1 : 
memberCount;

Review Comment:
   nit: A space is missing after the `-` sign.



##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec;
+import org.apache.kafka.coordinator.group.assignor.AssignmentSpec;
+import org.apache.kafka.coordinator.group.assignor.GroupAssignment;
+import org.apache.kafka.coordinator.group.assignor.MemberAssignment;
+import org.apache.kafka.coordinator.group.assignor.PartitionAssignor;
+import org.apache.kafka.coordinator.group.assignor.UniformAssignor;
+import org.apache.kafka.coordinator.group.consumer.Assignment;
+import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.VersionedMetadata;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 5)
+@Measurement(iterations = 5)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class TargetAssignmentBuilderBenchmark {
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;
+
+    String groupId = "benchmark-group";
+
+    private static final int groupEpoch = 0;
+
+    private PartitionAssignor partitionAssignor;
+
+    private Map<String, TopicMetadata> subscriptionMetadata = 
Collections.emptyMap();
+
+    private TargetAssignmentBuilder targetAssignmentBuilder;
+
+    private AssignmentSpec assignmentSpec;
+
+    private static final int numberOfRacks = 3;
+
+    private final List<String> allTopicNames = new ArrayList<>(topicCount);
+
+    private final List<Uuid> allTopicIds = new ArrayList<>(topicCount);
+
+    @Setup(Level.Trial)
+    public void setup() {
+        // For this benchmark we will use the Uniform Assignor
+        // and a group that has a homogeneous subscription model.
+        partitionAssignor = new UniformAssignor();
+        subscriptionMetadata = generateMockSubscriptionMetadata();
+        Map<String, ConsumerGroupMember> members = generateMockMembers();
+        Map<String, Assignment> existingTargetAssignment = 
generateMockInitialTargetAssignment();
+
+        // Add a new member to trigger a rebalance.
+        Set<String> subscribedTopics = new 
HashSet<>(subscriptionMetadata.keySet());
+        String rackId = isRackAware ? "rack" + (memberCount - 1) % 
numberOfRacks : "";
+        ConsumerGroupMember newMember = new 
ConsumerGroupMember.Builder("new-member")
+            .setSubscribedTopicNames(new ArrayList<>(subscribedTopics))
+            .setRackId(rackId)
+            .build();
+
+        targetAssignmentBuilder = new TargetAssignmentBuilder(groupId, 
groupEpoch, partitionAssignor)
+            .withMembers(members)
+            .withSubscriptionMetadata(subscriptionMetadata)
+            .withTargetAssignment(existingTargetAssignment)
+            .addOrUpdateMember(newMember.memberId(), newMember);
+    }
+
+    private Map<String, ConsumerGroupMember> generateMockMembers() {
+        Map<String, ConsumerGroupMember> members = new HashMap<>();
+
+        for (int i = 0; i < memberCount - 1; i++) {
+            Set<String> subscribedTopics;
+            subscribedTopics = new HashSet<>(allTopicNames);
+
+            String rackId = isRackAware ? "rack" + i % numberOfRacks : "" ;

Review Comment:
   ditto.



##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ClientSideAssignorBenchmark.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.CooperativeStickyAssignor;
+import org.apache.kafka.clients.consumer.RangeAssignor;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+
+import static java.lang.Integer.max;
+import static 
org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.DEFAULT_GENERATION;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 5)
+@Measurement(iterations = 5)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class ClientSideAssignorBenchmark {
+
+    public enum AssignorType {
+        RANGE(new RangeAssignor()),
+        COOPERATIVE_STICKY(new CooperativeStickyAssignor());
+
+        private final ConsumerPartitionAssignor assignor;
+
+        AssignorType(ConsumerPartitionAssignor assignor) {
+            this.assignor = assignor;
+        }
+
+        public ConsumerPartitionAssignor assignor() {
+            return assignor;
+        }
+    }
+
+    /**
+     * The subscription pattern followed by the members of the group.
+     *
+     * A subscription model is considered homogenous if all the members of the 
group
+     * are subscribed to the same set of topics, it is heterogeneous otherwise.
+     */
+    public enum SubscriptionModel {
+        HOMOGENEOUS, HETEROGENEOUS
+    }
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;
+
+    @Param({"HOMOGENEOUS", "HETEROGENEOUS"})
+    private SubscriptionModel subscriptionModel;
+
+    @Param({"RANGE", "COOPERATIVE_STICKY"})
+    private AssignorType assignorType;
+
+    @Param({"true", "false"})
+    private boolean simulateRebalanceTrigger;
+
+    private Map<String, ConsumerPartitionAssignor.Subscription> subscriptions 
= new HashMap<>();
+
+    private ConsumerPartitionAssignor.GroupSubscription groupSubscription;
+
+    private static final int NUMBER_OF_RACKS = 3;
+
+    private ConsumerPartitionAssignor assignor;
+
+    private Cluster metadata;
+
+    private final List<String> allTopicNames = new ArrayList<>(topicCount);
+
+    @Setup(Level.Trial)
+    public void setup() {
+        // Ensure there are enough racks and brokers for the replication 
factor.
+        if (NUMBER_OF_RACKS < 2) {
+            throw new IllegalArgumentException("Number of broker racks must be 
at least equal to 2.");
+        }
+
+        populateTopicMetadata();
+
+        addMemberSubscriptions();
+
+        assignor = assignorType.assignor();
+
+        if (simulateRebalanceTrigger) simulateRebalance();
+    }
+
+    private void populateTopicMetadata() {
+        List<PartitionInfo> partitions = new ArrayList<>();
+        int partitionsPerTopicCount = (memberCount * partitionsToMemberRatio) 
/ topicCount;
+
+        // Create nodes (brokers), one for each rack.
+        List<Node> nodes = new ArrayList<>(NUMBER_OF_RACKS);
+        for (int i = 0; i < NUMBER_OF_RACKS; i++) {
+            nodes.add(new Node(i, "", i, "rack" + i));
+        }
+
+        for (int i = 0; i < topicCount; i++) {
+            String topicName = "topic" + i;
+            allTopicNames.add(topicName);
+            partitions.addAll(partitionInfos(topicName, 
partitionsPerTopicCount, nodes));
+        }
+
+        metadata = new Cluster("test-cluster", nodes, partitions, 
Collections.emptySet(), Collections.emptySet());
+    }
+
+    private void addMemberSubscriptions() {
+        subscriptions.clear();
+        int topicCounter = 0;
+        Map<Integer, List<String>> tempMemberIndexToSubscriptions = new 
HashMap<>(memberCount);
+
+        // In the rebalance case, we will add the last member as a trigger.
+        // This is done to keep the total members count consistent with the 
input.
+        int numberOfMembers = simulateRebalanceTrigger ? memberCount -1 : 
memberCount;
+
+        for (int i = 0; i < numberOfMembers; i++) {
+            // When subscriptions are homogeneous, all members are assigned 
all topics.
+            if (subscriptionModel == SubscriptionModel.HOMOGENEOUS) {
+                String memberName = "member" + i;
+                subscriptions.put(memberName, subscription(allTopicNames, i));
+            } else {
+                List<String> subscribedTopics = Arrays.asList(
+                    allTopicNames.get(i % topicCount),
+                    allTopicNames.get((i+1) % topicCount)
+                );
+                topicCounter = max (topicCounter, ((i+1) % topicCount));
+                tempMemberIndexToSubscriptions.put(i, subscribedTopics);
+            }
+        }
+
+        int lastAssignedTopicIndex = topicCounter;
+        tempMemberIndexToSubscriptions.forEach((memberIndex, subscriptionList) 
-> {
+            String memberName = "member" + memberIndex;
+            if (lastAssignedTopicIndex < topicCount - 1) {
+                
subscriptionList.addAll(allTopicNames.subList(lastAssignedTopicIndex + 1, 
topicCount - 1));
+            }
+            subscriptions.put(memberName, subscription(allTopicNames, 
memberIndex));
+        });
+
+        groupSubscription = new 
ConsumerPartitionAssignor.GroupSubscription(subscriptions);
+    }
+
+    private List<PartitionInfo> partitionInfos(String topic, int 
numberOfPartitions, List<Node> nodes) {
+        // Create PartitionInfo for each partition.
+        // Replication factor is hardcoded to 2.
+        List<PartitionInfo> partitionInfos = new 
ArrayList<>(numberOfPartitions);
+        for (int i = 0; i < numberOfPartitions; i++) {
+            Node[] replicas = new Node[2];
+            for (int j = 0; j < 2; j++) {
+                // Assign nodes based on partition number to mimic 
mkMapOfPartitionRacks logic.
+                int nodeIndex = (i + j) % NUMBER_OF_RACKS;
+                replicas[j] = nodes.get(nodeIndex);
+            }
+            partitionInfos.add(new PartitionInfo(topic, i, replicas[0], 
replicas, replicas));
+        }
+
+        return partitionInfos;
+    }
+
+    private ConsumerPartitionAssignor.Subscription subscription(List<String> 
topics, int consumerIndex) {
+        Optional<String> rackId = rackId(consumerIndex);
+        return new ConsumerPartitionAssignor.Subscription(
+            topics,
+            null,
+            Collections.emptyList(),
+            DEFAULT_GENERATION,
+            rackId
+        );
+    }
+
+    private Optional<String> rackId(int index) {
+        return isRackAware ? Optional.of("rack" + index % NUMBER_OF_RACKS) : 
Optional.empty();
+    }
+
+    private ConsumerPartitionAssignor.Subscription 
subscriptionWithOwnedPartitions(
+        List<TopicPartition> ownedPartitions,
+        ConsumerPartitionAssignor.Subscription prevSubscription
+    ) {
+        return new ConsumerPartitionAssignor.Subscription(
+            prevSubscription.topics(),
+            null,
+            ownedPartitions,
+            DEFAULT_GENERATION,
+            prevSubscription.rackId()
+        );
+    }
+
+    private void simulateRebalance() {
+            ConsumerPartitionAssignor.GroupAssignment initialAssignment = 
assignor.assign(metadata, groupSubscription);
+            Map<String, ConsumerPartitionAssignor.Subscription> 
newSubscriptions = new HashMap<>();
+            subscriptions.forEach((member, subscription) ->
+                newSubscriptions.put(
+                    member,
+                    subscriptionWithOwnedPartitions(
+                        
initialAssignment.groupAssignment().get(member).partitions(),
+                        subscription
+                    )
+                )
+            );
+
+            // Add new member to trigger a reassignment.
+            newSubscriptions.put("newMember", subscription(
+                allTopicNames,
+                memberCount - 1
+            ));
+
+            this.subscriptions = newSubscriptions;

Review Comment:
   We can remove `this.`.



##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ClientSideAssignorBenchmark.java:
##########
@@ -0,0 +1,262 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor;
+import org.apache.kafka.clients.consumer.CooperativeStickyAssignor;
+import org.apache.kafka.clients.consumer.RangeAssignor;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+
+import static java.lang.Integer.max;
+import static 
org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.DEFAULT_GENERATION;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 5)
+@Measurement(iterations = 5)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class ClientSideAssignorBenchmark {
+
+    public enum AssignorType {
+        RANGE(new RangeAssignor()),
+        COOPERATIVE_STICKY(new CooperativeStickyAssignor());
+
+        private final ConsumerPartitionAssignor assignor;
+
+        AssignorType(ConsumerPartitionAssignor assignor) {
+            this.assignor = assignor;
+        }
+
+        public ConsumerPartitionAssignor assignor() {
+            return assignor;
+        }
+    }
+
+    /**
+     * The subscription pattern followed by the members of the group.
+     *
+     * A subscription model is considered homogenous if all the members of the 
group
+     * are subscribed to the same set of topics, it is heterogeneous otherwise.
+     */
+    public enum SubscriptionModel {
+        HOMOGENEOUS, HETEROGENEOUS
+    }
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;
+
+    @Param({"HOMOGENEOUS", "HETEROGENEOUS"})
+    private SubscriptionModel subscriptionModel;
+
+    @Param({"RANGE", "COOPERATIVE_STICKY"})
+    private AssignorType assignorType;
+
+    @Param({"true", "false"})
+    private boolean simulateRebalanceTrigger;
+
+    private Map<String, ConsumerPartitionAssignor.Subscription> subscriptions 
= new HashMap<>();
+
+    private ConsumerPartitionAssignor.GroupSubscription groupSubscription;
+
+    private static final int NUMBER_OF_RACKS = 3;
+
+    private ConsumerPartitionAssignor assignor;
+
+    private Cluster metadata;
+
+    private final List<String> allTopicNames = new ArrayList<>(topicCount);
+
+    @Setup(Level.Trial)
+    public void setup() {
+        // Ensure there are enough racks and brokers for the replication 
factor.
+        if (NUMBER_OF_RACKS < 2) {
+            throw new IllegalArgumentException("Number of broker racks must be 
at least equal to 2.");
+        }
+
+        populateTopicMetadata();
+
+        addMemberSubscriptions();
+
+        assignor = assignorType.assignor();
+
+        if (simulateRebalanceTrigger) simulateRebalance();
+    }
+
+    private void populateTopicMetadata() {
+        List<PartitionInfo> partitions = new ArrayList<>();
+        int partitionsPerTopicCount = (memberCount * partitionsToMemberRatio) 
/ topicCount;
+
+        // Create nodes (brokers), one for each rack.
+        List<Node> nodes = new ArrayList<>(NUMBER_OF_RACKS);
+        for (int i = 0; i < NUMBER_OF_RACKS; i++) {
+            nodes.add(new Node(i, "", i, "rack" + i));
+        }
+
+        for (int i = 0; i < topicCount; i++) {
+            String topicName = "topic" + i;
+            allTopicNames.add(topicName);
+            partitions.addAll(partitionInfos(topicName, 
partitionsPerTopicCount, nodes));
+        }
+
+        metadata = new Cluster("test-cluster", nodes, partitions, 
Collections.emptySet(), Collections.emptySet());
+    }
+
+    private void addMemberSubscriptions() {
+        subscriptions.clear();

Review Comment:
   Do we have to clear it because the instance variables are not reset between 
run? If so, it would be better to not have initial values for them and to 
initial them in `setup()`.



##########
jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.jmh.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.assignor.AssignmentMemberSpec;
+import org.apache.kafka.coordinator.group.assignor.AssignmentSpec;
+import org.apache.kafka.coordinator.group.assignor.GroupAssignment;
+import org.apache.kafka.coordinator.group.assignor.MemberAssignment;
+import org.apache.kafka.coordinator.group.assignor.PartitionAssignor;
+import org.apache.kafka.coordinator.group.assignor.UniformAssignor;
+import org.apache.kafka.coordinator.group.consumer.Assignment;
+import org.apache.kafka.coordinator.group.consumer.ConsumerGroupMember;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TargetAssignmentBuilder;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.VersionedMetadata;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.BenchmarkMode;
+import org.openjdk.jmh.annotations.Fork;
+import org.openjdk.jmh.annotations.Level;
+import org.openjdk.jmh.annotations.Measurement;
+import org.openjdk.jmh.annotations.Mode;
+import org.openjdk.jmh.annotations.OutputTimeUnit;
+import org.openjdk.jmh.annotations.Param;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.Setup;
+import org.openjdk.jmh.annotations.State;
+import org.openjdk.jmh.annotations.Threads;
+import org.openjdk.jmh.annotations.Warmup;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+@State(Scope.Benchmark)
+@Fork(value = 1)
+@Warmup(iterations = 5)
+@Measurement(iterations = 5)
+@BenchmarkMode(Mode.AverageTime)
+@OutputTimeUnit(TimeUnit.MILLISECONDS)
+public class TargetAssignmentBuilderBenchmark {
+
+    @Param({"100", "500", "1000", "5000", "10000"})
+    private int memberCount;
+
+    @Param({"5", "10", "50"})
+    private int partitionsToMemberRatio;
+
+    @Param({"10", "100", "1000"})
+    private int topicCount;
+
+    @Param({"true", "false"})
+    private boolean isRackAware;

Review Comment:
   Do we need this? I suppose that we don't.



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