[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-20 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1332072827


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * The optimized uniform assignment builder is used to generate the target 
assignment for a consumer group with
+ * all its members subscribed to the same set of topics.
+ * It is optimized since the assignment can be done in fewer, less complicated 
steps compared to when
+ * the subscriptions are different across the members.
+ *
+ * Assignments are done according to the following principles:
+ *
+ *
+ *  Balance:  Ensure partitions are distributed equally among all 
members.
+ *The difference in assignments sizes between any two 
members
+ *should not exceed one partition. 
+ *  Rack Matching:When feasible, aim to assign partitions to members
+ *located on the same rack thus avoiding cross-zone 
traffic. 
+ *  Stickiness:   Minimize partition movements among members by 
retaining
+ *as much of the existing assignment as possible. 
+ *
+ * The assignment builder prioritizes the properties in the following order:
+ *  Balance > Rack Matching > Stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger LOG = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+/**
+ * The assignment specification which includes member metadata.
+ */
+private final AssignmentSpec assignmentSpec;
+/**
+ * The topic and partition metadata describer.
+ */
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+/**
+ * The set of topic Ids that the consumer group is subscribed to.
+ */
+private final Set subscriptionIds;
+/**
+ * Rack information and helper methods.
+ */
+private final RackInfo rackInfo;
+/**
+ * The number of members to receive an extra partition beyond the minimum 
quota.
+ * Minimum Quota = Total Partitions / Total Members
+ * Example: If there are 11 partitions to be distributed among 3 members,
+ *  each member gets 3 (11 / 3) [minQuota] partitions and 2 (11 % 
3) members get an extra partition.
+ */
+private int remainingMembersToGetAnExtraPartition;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
minimum quota,
+ * including members that are eligible to receive an extra partition.
+ */
+private final Map potentiallyUnfilledMembers;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
full quota.
+ * Full quota = minQuota + one extra partition (if applicable).
+ */
+private Map unfilledMembers;
+/**
+ * The partitions that still need to be assigned.
+ * Initially this contains all the subscribed topics' partitions.
+ */
+private List unassignedPartitions;
+/**
+ * The target assignment.
+ */
+private final Map targetAssignment;
+/**
+ * Tracks the existing owner of each partition.
+ * Only populated when the rack awareness strategy is used.
+ */
+private final Map currentPartitionOwners;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-15 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1326838235


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * The optimized uniform assignment builder is used to generate the target 
assignment for a consumer group with
+ * all its members subscribed to the same set of topics.
+ * It is optimized since the assignment can be done in fewer, less complicated 
steps compared to when
+ * the subscriptions are different across the members.
+ *
+ * Assignments are done according to the following principles:
+ *
+ *
+ *  Balance:  Ensure partitions are distributed equally among all 
members.
+ *The difference in assignments sizes between any two 
members
+ *should not exceed one partition. 
+ *  Rack Matching:When feasible, aim to assign partitions to members
+ *located on the same rack thus avoiding cross-zone 
traffic. 
+ *  Stickiness:   Minimize partition movements among members by 
retaining
+ *as much of the existing assignment as possible. 
+ *
+ * The assignment builder prioritizes the properties in the following order:
+ *  Balance > Rack Matching > Stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger LOG = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+/**
+ * The assignment specification which includes member metadata.
+ */
+private final AssignmentSpec assignmentSpec;
+/**
+ * The topic and partition metadata describer.
+ */
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+/**
+ * The set of topic Ids that the consumer group is subscribed to.
+ */
+private final Set subscriptionIds;
+/**
+ * Rack information and helper methods.
+ */
+private final RackInfo rackInfo;
+/**
+ * The number of members to receive an extra partition beyond the minimum 
quota.
+ * Minimum Quota = Total Partitions / Total Members
+ * Example: If there are 11 partitions to be distributed among 3 members,
+ *  each member gets 3 (11 / 3) [minQuota] partitions and 2 (11 % 
3) members get an extra partition.
+ */
+private int remainingMembersToGetAnExtraPartition;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
minimum quota,
+ * including members that are eligible to receive an extra partition.
+ */
+private final Map potentiallyUnfilledMembers;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
full quota.
+ * Full quota = minQuota + one extra partition (if applicable).
+ */
+private Map unfilledMembers;
+/**
+ * The partitions that still need to be assigned.
+ * Initially this contains all the subscribed topics' partitions.
+ */
+private List unassignedPartitions;
+/**
+ * The target assignment.
+ */
+private final Map targetAssignment;
+/**
+ * Tracks the existing owner of each partition.
+ * Only populated when the rack awareness strategy is used.
+ */
+private final Map currentPartitionOwners;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-14 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1326826466


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * The optimized uniform assignment builder is used to generate the target 
assignment for a consumer group with
+ * all its members subscribed to the same set of topics.
+ * It is optimized since the assignment can be done in fewer, less complicated 
steps compared to when
+ * the subscriptions are different across the members.
+ *
+ * Assignments are done according to the following principles:
+ *
+ *
+ *  Balance:  Ensure partitions are distributed equally among all 
members.
+ *The difference in assignments sizes between any two 
members
+ *should not exceed one partition. 
+ *  Rack Matching:When feasible, aim to assign partitions to members
+ *located on the same rack thus avoiding cross-zone 
traffic. 
+ *  Stickiness:   Minimize partition movements among members by 
retaining
+ *as much of the existing assignment as possible. 
+ *
+ * The assignment builder prioritizes the properties in the following order:
+ *  Balance > Rack Matching > Stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger LOG = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+/**
+ * The assignment specification which includes member metadata.
+ */
+private final AssignmentSpec assignmentSpec;
+/**
+ * The topic and partition metadata describer.
+ */
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+/**
+ * The set of topic Ids that the consumer group is subscribed to.
+ */
+private final Set subscriptionIds;
+/**
+ * Rack information and helper methods.
+ */
+private final RackInfo rackInfo;
+/**
+ * The number of members to receive an extra partition beyond the minimum 
quota.
+ * Minimum Quota = Total Partitions / Total Members
+ * Example: If there are 11 partitions to be distributed among 3 members,
+ *  each member gets 3 (11 / 3) [minQuota] partitions and 2 (11 % 
3) members get an extra partition.
+ */
+private int remainingMembersToGetAnExtraPartition;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
minimum quota,
+ * including members that are eligible to receive an extra partition.
+ */
+private final Map potentiallyUnfilledMembers;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
full quota.
+ * Full quota = minQuota + one extra partition (if applicable).
+ */
+private Map unfilledMembers;
+/**
+ * The partitions that still need to be assigned.
+ * Initially this contains all the subscribed topics' partitions.
+ */
+private List unassignedPartitions;
+/**
+ * The target assignment.
+ */
+private final Map targetAssignment;
+/**
+ * Tracks the existing owner of each partition.
+ * Only populated when the rack awareness strategy is used.
+ */
+private final Map currentPartitionOwners;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-14 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1326819932


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * The optimized uniform assignment builder is used to generate the target 
assignment for a consumer group with
+ * all its members subscribed to the same set of topics.
+ * It is optimized since the assignment can be done in fewer, less complicated 
steps compared to when
+ * the subscriptions are different across the members.
+ *
+ * Assignments are done according to the following principles:
+ *
+ *
+ *  Balance:  Ensure partitions are distributed equally among all 
members.
+ *The difference in assignments sizes between any two 
members
+ *should not exceed one partition. 
+ *  Rack Matching:When feasible, aim to assign partitions to members
+ *located on the same rack thus avoiding cross-zone 
traffic. 
+ *  Stickiness:   Minimize partition movements among members by 
retaining
+ *as much of the existing assignment as possible. 
+ *
+ * The assignment builder prioritizes the properties in the following order:
+ *  Balance > Rack Matching > Stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger LOG = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+/**
+ * The assignment specification which includes member metadata.
+ */
+private final AssignmentSpec assignmentSpec;
+/**
+ * The topic and partition metadata describer.
+ */
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+/**
+ * The set of topic Ids that the consumer group is subscribed to.
+ */
+private final Set subscriptionIds;
+/**
+ * Rack information and helper methods.
+ */
+private final RackInfo rackInfo;
+/**
+ * The number of members to receive an extra partition beyond the minimum 
quota.
+ * Minimum Quota = Total Partitions / Total Members
+ * Example: If there are 11 partitions to be distributed among 3 members,
+ *  each member gets 3 (11 / 3) [minQuota] partitions and 2 (11 % 
3) members get an extra partition.
+ */
+private int remainingMembersToGetAnExtraPartition;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
minimum quota,
+ * including members that are eligible to receive an extra partition.
+ */
+private final Map potentiallyUnfilledMembers;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
full quota.
+ * Full quota = minQuota + one extra partition (if applicable).
+ */
+private Map unfilledMembers;
+/**
+ * The partitions that still need to be assigned.
+ * Initially this contains all the subscribed topics' partitions.
+ */
+private List unassignedPartitions;
+/**
+ * The target assignment.
+ */
+private final Map targetAssignment;
+/**
+ * Tracks the existing owner of each partition.
+ * Only populated when the rack awareness strategy is used.
+ */
+private final Map currentPartitionOwners;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-14 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1326818424


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/AbstractUniformAssignmentBuilder.java:
##
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The assignment builder is used to construct the target assignment based on 
the members' subscriptions.
+ *
+ * This class contains common utility methods and a class for obtaining and 
storing rack information.
+ */
+public abstract class AbstractUniformAssignmentBuilder {
+protected abstract GroupAssignment buildAssignment();
+
+/**
+ * Determines if rack-aware assignment is appropriate based on the 
provided rack information.
+ *
+ * @param memberRacks   Racks where members are located.
+ * @param partitionRacksRacks where partitions are located.
+ * @param racksPerPartition Map of partitions to their associated 
racks.
+ *
+ * @return {@code true} if rack-aware assignment should be applied; {@code 
false} otherwise.
+ */
+protected static boolean useRackAwareAssignment(
+Set memberRacks,
+Set partitionRacks,
+Map> racksPerPartition
+) {
+if (memberRacks.isEmpty() || Collections.disjoint(memberRacks, 
partitionRacks))
+return false;
+else {
+return 
!racksPerPartition.values().stream().allMatch(partitionRacks::equals);
+}
+}
+
+/**
+ * Adds the topic's partition to the member's target assignment.
+ */
+protected static void addPartitionToAssignment(
+int partition,
+Uuid topicId,
+String memberId,
+Map targetAssignment

Review Comment:
   Makes sense, I put the order based on the method name like add partition to 
assignment so first partition then which topic then whose assignment and then 
which assignment



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-14 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1326817433


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/AbstractUniformAssignmentBuilder.java:
##
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The assignment builder is used to construct the target assignment based on 
the members' subscriptions.
+ *
+ * This class contains common utility methods and a class for obtaining and 
storing rack information.
+ */
+public abstract class AbstractUniformAssignmentBuilder {
+protected abstract GroupAssignment buildAssignment();
+
+/**
+ * Determines if rack-aware assignment is appropriate based on the 
provided rack information.
+ *
+ * @param memberRacks   Racks where members are located.
+ * @param partitionRacksRacks where partitions are located.
+ * @param racksPerPartition Map of partitions to their associated 
racks.
+ *
+ * @return {@code true} if rack-aware assignment should be applied; {@code 
false} otherwise.
+ */
+protected static boolean useRackAwareAssignment(
+Set memberRacks,
+Set partitionRacks,
+Map> racksPerPartition
+) {
+if (memberRacks.isEmpty() || Collections.disjoint(memberRacks, 
partitionRacks))
+return false;
+else {
+return 
!racksPerPartition.values().stream().allMatch(partitionRacks::equals);
+}
+}
+
+/**
+ * Adds the topic's partition to the member's target assignment.
+ */
+protected static void addPartitionToAssignment(
+int partition,
+Uuid topicId,
+String memberId,
+Map targetAssignment
+) {
+targetAssignment.get(memberId)
+.targetPartitions()
+.computeIfAbsent(topicId, __ -> new HashSet<>())
+.add(partition);
+}
+
+/**
+ * Constructs a list of {@code TopicIdPartition} for each topic Id based 
on its partition count.
+ *
+ * @param allTopicIds   The subscribed topic Ids.
+ * @param subscribedTopicDescriber  Utility to fetch the partition 
count for a given topic.
+ *
+ * @return List of sorted {@code TopicIdPartition} for all provided topic 
Ids.
+ */
+protected static List allTopicIdPartitions(
+Collection allTopicIds,
+SubscribedTopicDescriber subscribedTopicDescriber
+) {
+List allTopicIdPartitions = new ArrayList<>();
+// Sorted so that partitions from each topic can be distributed 
amongst its subscribers equally.
+allTopicIds.stream().sorted().forEach(topic ->
+IntStream.range(0, subscribedTopicDescriber.numPartitions(topic))
+.forEach(i -> allTopicIdPartitions.add(new 
TopicIdPartition(topic, i))
+)

Review Comment:
   I wasn't sure where it went tbh xD



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-14 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1326815225


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java:
##
@@ -0,0 +1,1090 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.junit.jupiter.api.Test;
+
+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.TreeMap;
+
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static 
org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class OptimizedUniformAssignmentBuilderTest {
+private final UniformAssignor assignor = new UniformAssignor();
+private final Uuid topic1Uuid = Uuid.fromString("T1-A4s3VTwiI5CTbEp6POw");
+private final Uuid topic2Uuid = Uuid.fromString("T2-B4s3VTwiI5YHbPp6YUe");
+private final Uuid topic3Uuid = Uuid.fromString("T3-CU8fVTLCz5YMkLoDQsa");
+private final String topic1Name = "topic1";
+private final String topic2Name = "topic2";
+private final String topic3Name = "topic3";
+private final String memberA = "A";
+private final String memberB = "B";
+private final String memberC = "C";
+
+@Test
+public void testOneMemberNoTopicSubscription() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.emptyList(),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void testOneMemberSubscribedToNonexistentTopic() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.singletonList(topic2Uuid),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+
+assertThrows(PartitionAssignorException.class,
+() -> assignor.assign(assignmentSpec, subscribedTopicMetadata));
+}
+
+@Test
+public void testFirstAssignmentTwoMembersTwoTopicsNoMemberRacks() {
+Map topicMetadata = new HashMap<>();
+topicMetadata.put(topic1Uuid, new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+));
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-11 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1321035644


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

Review Comment:
   yeah that's true I agree, I wasn't sure what else to call this one, okay I 
will add a new PR for this.



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-11 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1321033941


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * The optimized uniform assignment builder is used to generate the target 
assignment for a consumer group with
+ * all its members subscribed to the same set of topics.
+ * It is optimized since the assignment can be done in fewer, less complicated 
steps compared to when
+ * the subscriptions are different across the members.
+ *
+ * Assignments are done according to the following principles:
+ *
+ *
+ *  Balance:  Ensure partitions are distributed equally among all 
members.
+ *The difference in assignments sizes between any two 
members
+ *should not exceed one partition. 
+ *  Rack Matching:When feasible, aim to assign partitions to members
+ *located on the same rack thus avoiding cross-zone 
traffic. 
+ *  Stickiness:   Minimize partition movements among members by 
retaining
+ *as much of the existing assignment as possible. 
+ *
+ * The assignment builder prioritizes the properties in the following order:
+ *  Balance > Rack Matching > Stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger LOG = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+/**
+ * The assignment specification which includes member metadata.
+ */
+private final AssignmentSpec assignmentSpec;
+/**
+ * The topic and partition metadata describer.
+ */
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+/**
+ * The set of topic Ids that the consumer group is subscribed to.
+ */
+private final Set subscriptionIds;
+/**
+ * Rack information and helper methods.
+ */
+private final RackInfo rackInfo;
+/**
+ * The number of members to receive an extra partition beyond the minimum 
quota.
+ * Minimum Quota = Total Partitions / Total Members
+ * Example: If there are 11 partitions to be distributed among 3 members,
+ *  each member gets 3 (11 / 3) [minQuota] partitions and 2 (11 % 
3) members get an extra partition.
+ */
+private int remainingMembersToGetAnExtraPartition;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
minimum quota,
+ * including members that are eligible to receive an extra partition.
+ */
+private final Map potentiallyUnfilledMembers;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
full quota.
+ * Full quota = minQuota + one extra partition (if applicable).
+ */
+private Map unfilledMembers;
+/**
+ * The partitions that still need to be assigned.
+ * Initially this contains all the subscribed topics' partitions.
+ */
+private List unassignedPartitions;
+/**
+ * The target assignment.
+ */
+private final Map targetAssignment;
+/**
+ * Tracks the existing owner of each partition.
+ * Only populated when the rack awareness strategy is used.
+ */
+private final Map currentPartitionOwners;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-10 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1321024671


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * The optimized uniform assignment builder is used to generate the target 
assignment for a consumer group with
+ * all its members subscribed to the same set of topics.
+ * It is optimized since the assignment can be done in fewer, less complicated 
steps compared to when
+ * the subscriptions are different across the members.
+ *
+ * Assignments are done according to the following principles:
+ *
+ *
+ *  Balance:  Ensure partitions are distributed equally among all 
members.
+ *The difference in assignments sizes between any two 
members
+ *should not exceed one partition. 
+ *  Rack Matching:When feasible, aim to assign partitions to members
+ *located on the same rack thus avoiding cross-zone 
traffic. 
+ *  Stickiness:   Minimize partition movements among members by 
retaining
+ *as much of the existing assignment as possible. 
+ *
+ * The assignment builder prioritizes the properties in the following order:
+ *  Balance > Rack Matching > Stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger LOG = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+/**
+ * The assignment specification which includes member metadata.
+ */
+private final AssignmentSpec assignmentSpec;
+/**
+ * The topic and partition metadata describer.
+ */
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+/**
+ * The set of topic Ids that the consumer group is subscribed to.
+ */
+private final Set subscriptionIds;
+/**
+ * Rack information and helper methods.
+ */
+private final RackInfo rackInfo;
+/**
+ * The number of members to receive an extra partition beyond the minimum 
quota.
+ * Minimum Quota = Total Partitions / Total Members
+ * Example: If there are 11 partitions to be distributed among 3 members,
+ *  each member gets 3 (11 / 3) [minQuota] partitions and 2 (11 % 
3) members get an extra partition.
+ */
+private int remainingMembersToGetAnExtraPartition;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
minimum quota,
+ * including members that are eligible to receive an extra partition.
+ */
+private final Map potentiallyUnfilledMembers;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
full quota.
+ * Full quota = minQuota + one extra partition (if applicable).
+ */
+private Map unfilledMembers;
+/**
+ * The partitions that still need to be assigned.
+ * Initially this contains all the subscribed topics' partitions.
+ */
+private List unassignedPartitions;
+/**
+ * The target assignment.
+ */
+private final Map targetAssignment;
+/**
+ * Tracks the existing owner of each partition.
+ * Only populated when the rack awareness strategy is used.
+ */
+private final Map currentPartitionOwners;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-10 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1321023849


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * The optimized uniform assignment builder is used to generate the target 
assignment for a consumer group with
+ * all its members subscribed to the same set of topics.
+ * It is optimized since the assignment can be done in fewer, less complicated 
steps compared to when
+ * the subscriptions are different across the members.
+ *
+ * Assignments are done according to the following principles:
+ *
+ *
+ *  Balance:  Ensure partitions are distributed equally among all 
members.
+ *The difference in assignments sizes between any two 
members
+ *should not exceed one partition. 
+ *  Rack Matching:When feasible, aim to assign partitions to members
+ *located on the same rack thus avoiding cross-zone 
traffic. 
+ *  Stickiness:   Minimize partition movements among members by 
retaining
+ *as much of the existing assignment as possible. 
+ *
+ * The assignment builder prioritizes the properties in the following order:
+ *  Balance > Rack Matching > Stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger LOG = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+/**
+ * The assignment specification which includes member metadata.
+ */
+private final AssignmentSpec assignmentSpec;
+/**
+ * The topic and partition metadata describer.
+ */
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+/**
+ * The set of topic Ids that the consumer group is subscribed to.
+ */
+private final Set subscriptionIds;
+/**
+ * Rack information and helper methods.
+ */
+private final RackInfo rackInfo;
+/**
+ * The number of members to receive an extra partition beyond the minimum 
quota.
+ * Minimum Quota = Total Partitions / Total Members
+ * Example: If there are 11 partitions to be distributed among 3 members,
+ *  each member gets 3 (11 / 3) [minQuota] partitions and 2 (11 % 
3) members get an extra partition.
+ */
+private int remainingMembersToGetAnExtraPartition;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
minimum quota,
+ * including members that are eligible to receive an extra partition.
+ */
+private final Map potentiallyUnfilledMembers;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
full quota.
+ * Full quota = minQuota + one extra partition (if applicable).
+ */
+private Map unfilledMembers;
+/**
+ * The partitions that still need to be assigned.
+ * Initially this contains all the subscribed topics' partitions.
+ */
+private List unassignedPartitions;
+/**
+ * The target assignment.
+ */
+private final Map targetAssignment;
+/**
+ * Tracks the existing owner of each partition.
+ * Only populated when the rack awareness strategy is used.
+ */
+private final Map currentPartitionOwners;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-10 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1321023603


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * The optimized uniform assignment builder is used to generate the target 
assignment for a consumer group with
+ * all its members subscribed to the same set of topics.
+ * It is optimized since the assignment can be done in fewer, less complicated 
steps compared to when
+ * the subscriptions are different across the members.
+ *
+ * Assignments are done according to the following principles:
+ *
+ *
+ *  Balance:  Ensure partitions are distributed equally among all 
members.
+ *The difference in assignments sizes between any two 
members
+ *should not exceed one partition. 
+ *  Rack Matching:When feasible, aim to assign partitions to members
+ *located on the same rack thus avoiding cross-zone 
traffic. 
+ *  Stickiness:   Minimize partition movements among members by 
retaining
+ *as much of the existing assignment as possible. 
+ *
+ * The assignment builder prioritizes the properties in the following order:
+ *  Balance > Rack Matching > Stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger LOG = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+/**
+ * The assignment specification which includes member metadata.
+ */
+private final AssignmentSpec assignmentSpec;
+/**
+ * The topic and partition metadata describer.
+ */
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+/**
+ * The set of topic Ids that the consumer group is subscribed to.
+ */
+private final Set subscriptionIds;
+/**
+ * Rack information and helper methods.
+ */
+private final RackInfo rackInfo;
+/**
+ * The number of members to receive an extra partition beyond the minimum 
quota.
+ * Minimum Quota = Total Partitions / Total Members
+ * Example: If there are 11 partitions to be distributed among 3 members,
+ *  each member gets 3 (11 / 3) [minQuota] partitions and 2 (11 % 
3) members get an extra partition.
+ */
+private int remainingMembersToGetAnExtraPartition;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
minimum quota,
+ * including members that are eligible to receive an extra partition.
+ */
+private final Map potentiallyUnfilledMembers;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
full quota.
+ * Full quota = minQuota + one extra partition (if applicable).
+ */
+private Map unfilledMembers;
+/**
+ * The partitions that still need to be assigned.
+ * Initially this contains all the subscribed topics' partitions.
+ */
+private List unassignedPartitions;
+/**
+ * The target assignment.
+ */
+private final Map targetAssignment;
+/**
+ * Tracks the existing owner of each partition.
+ * Only populated when the rack awareness strategy is used.
+ */
+private final Map currentPartitionOwners;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-08 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1320220933


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * The optimized uniform assignment builder is used to generate the target 
assignment for a consumer group with
+ * all its members subscribed to the same set of topics.
+ * It is optimized since the assignment can be done in fewer, less complicated 
steps compared to when
+ * the subscriptions are different across the members.
+ *
+ * Assignments are done according to the following principles:
+ *
+ *
+ *  Balance:  Ensure partitions are distributed equally among all 
members.
+ *The difference in assignments sizes between any two 
members
+ *should not exceed one partition. 
+ *  Rack Matching:When feasible, aim to assign partitions to members
+ *located on the same rack thus avoiding cross-zone 
traffic. 
+ *  Stickiness:   Minimize partition movements among members by 
retaining
+ *as much of the existing assignment as possible. 
+ *
+ * The assignment builder prioritizes the properties in the following order:
+ *  Balance > Rack Matching > Stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger LOG = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+/**
+ * The assignment specification which includes member metadata.
+ */
+private final AssignmentSpec assignmentSpec;
+/**
+ * The topic and partition metadata describer.
+ */
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+/**
+ * The set of topic Ids that the consumer group is subscribed to.
+ */
+private final Set subscriptionIds;
+/**
+ * Rack information and helper methods.
+ */
+private final RackInfo rackInfo;
+/**
+ * The number of members to receive an extra partition beyond the minimum 
quota.
+ * Minimum Quota = Total Partitions / Total Members
+ * Example: If there are 11 partitions to be distributed among 3 members,
+ *  each member gets 3 (11 / 3) [minQuota] partitions and 2 (11 % 
3) members get an extra partition.
+ */
+private int remainingMembersToGetAnExtraPartition;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
minimum quota,
+ * including members that are eligible to receive an extra partition.
+ */
+private final Map potentiallyUnfilledMembers;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
full quota.
+ * Full quota = minQuota + one extra partition (if applicable).
+ */
+private Map unfilledMembers;
+/**
+ * The partitions that still need to be assigned.
+ * Initially this contains all the subscribed topics' partitions.
+ */
+private List unassignedPartitions;
+/**
+ * The target assignment.
+ */
+private final Map targetAssignment;
+/**
+ * Tracks the existing owner of each partition.
+ * Only populated when the rack awareness strategy is used.
+ */
+private final Map currentPartitionOwners;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-08 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1320220040


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * The optimized uniform assignment builder is used to generate the target 
assignment for a consumer group with
+ * all its members subscribed to the same set of topics.
+ * It is optimized since the assignment can be done in fewer, less complicated 
steps compared to when
+ * the subscriptions are different across the members.
+ *
+ * Assignments are done according to the following principles:
+ *
+ *
+ *  Balance:  Ensure partitions are distributed equally among all 
members.
+ *The difference in assignments sizes between any two 
members
+ *should not exceed one partition. 
+ *  Rack Matching:When feasible, aim to assign partitions to members
+ *located on the same rack thus avoiding cross-zone 
traffic. 
+ *  Stickiness:   Minimize partition movements among members by 
retaining
+ *as much of the existing assignment as possible. 
+ *
+ * The assignment builder prioritizes the properties in the following order:
+ *  Balance > Rack Matching > Stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger LOG = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+/**
+ * The assignment specification which includes member metadata.
+ */
+private final AssignmentSpec assignmentSpec;
+/**
+ * The topic and partition metadata describer.
+ */
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+/**
+ * The set of topic Ids that the consumer group is subscribed to.
+ */
+private final Set subscriptionIds;
+/**
+ * Rack information and helper methods.
+ */
+private final RackInfo rackInfo;
+/**
+ * The number of members to receive an extra partition beyond the minimum 
quota.
+ * Minimum Quota = Total Partitions / Total Members
+ * Example: If there are 11 partitions to be distributed among 3 members,
+ *  each member gets 3 (11 / 3) [minQuota] partitions and 2 (11 % 
3) members get an extra partition.
+ */
+private int remainingMembersToGetAnExtraPartition;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
minimum quota,
+ * including members that are eligible to receive an extra partition.
+ */
+private final Map potentiallyUnfilledMembers;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
full quota.
+ * Full quota = minQuota + one extra partition (if applicable).
+ */
+private Map unfilledMembers;
+/**
+ * The partitions that still need to be assigned.
+ * Initially this contains all the subscribed topics' partitions.
+ */
+private List unassignedPartitions;
+/**
+ * The target assignment.
+ */
+private final Map targetAssignment;
+/**
+ * Tracks the existing owner of each partition.
+ * Only populated when the rack awareness strategy is used.
+ */
+private final Map currentPartitionOwners;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-08 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1320217221


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * The optimized uniform assignment builder is used to generate the target 
assignment for a consumer group with
+ * all its members subscribed to the same set of topics.
+ * It is optimized since the assignment can be done in fewer, less complicated 
steps compared to when
+ * the subscriptions are different across the members.
+ *
+ * Assignments are done according to the following principles:
+ *
+ *
+ *  Balance:  Ensure partitions are distributed equally among all 
members.
+ *The difference in assignments sizes between any two 
members
+ *should not exceed one partition. 
+ *  Rack Matching:When feasible, aim to assign partitions to members
+ *located on the same rack thus avoiding cross-zone 
traffic. 
+ *  Stickiness:   Minimize partition movements among members by 
retaining
+ *as much of the existing assignment as possible. 
+ *
+ * The assignment builder prioritizes the properties in the following order:
+ *  Balance > Rack Matching > Stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger LOG = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+/**
+ * The assignment specification which includes member metadata.
+ */
+private final AssignmentSpec assignmentSpec;
+/**
+ * The topic and partition metadata describer.
+ */
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+/**
+ * The set of topic Ids that the consumer group is subscribed to.
+ */
+private final Set subscriptionIds;
+/**
+ * Rack information and helper methods.
+ */
+private final RackInfo rackInfo;
+/**
+ * The number of members to receive an extra partition beyond the minimum 
quota.
+ * Minimum Quota = Total Partitions / Total Members
+ * Example: If there are 11 partitions to be distributed among 3 members,
+ *  each member gets 3 (11 / 3) [minQuota] partitions and 2 (11 % 
3) members get an extra partition.
+ */
+private int remainingMembersToGetAnExtraPartition;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
minimum quota,
+ * including members that are eligible to receive an extra partition.
+ */
+private final Map potentiallyUnfilledMembers;
+/**
+ * Members mapped to the remaining number of partitions needed to meet the 
full quota.
+ * Full quota = minQuota + one extra partition (if applicable).
+ */
+private Map unfilledMembers;
+/**
+ * The partitions that still need to be assigned.
+ * Initially this contains all the subscribed topics' partitions.
+ */
+private List unassignedPartitions;
+/**
+ * The target assignment.
+ */
+private final Map targetAssignment;
+/**
+ * Tracks the existing owner of each partition.
+ * Only populated when the rack awareness strategy is used.
+ */
+private final Map currentPartitionOwners;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-06 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1317676113


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *  balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final Set subscriptionIds;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionIds = new 
HashSet<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+this.rackInfo = new RackInfo(assignmentSpec, subscribedTopicDescriber, 
subscriptionIds);
+this.potentiallyUnfilledMembers = new HashMap<>();
+this.unfilledMembers = new HashMap<>();
+this.newAssignment = new HashMap<>();
+this.useRackAwareStrategy = rackInfo.useRackStrategy;
+// Without rack-aware strategy, tracking current owners of unassigned 
partitions is unnecessary
+// as all sticky partitions are retained until a member meets its 
quota.
+this.currentPartitionOwners = useRackAwareStrategy ? new HashMap<>() : 
Collections.emptyMap();
+}
+
+/**
+ * Here's the step-by-step breakdown of the assignment process:
+ *
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its owner, track it for 
future use.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-06 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1317669213


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

Review Comment:
   Ohh my bad I got this question before so I instinctively replied, I realized 
I was thinking of the kafka common topicIdPartition class but yeah this seems 
usable, would we need a new PR to move the file though?



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-06 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1317669213


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

Review Comment:
   Ohh my bad I got this question before so I instinctively replied, I didn't 
know this class existed, yeah this seems usable, would we need a new PR to move 
the file though?



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-06 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1317663852


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##
@@ -0,0 +1,288 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * 
+ * 
+ *  Optimized Uniform Assignment Builder:  This strategy is 
used when all members have subscribed
+ * to the same set of topics.
+ * 
+ * 
+ *  General Uniform Assignment Builder:  This strategy is used 
when members have varied topic
+ * subscriptions.
+ * 
+ * 
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+@Override
+public String name() {
+return UNIFORM_ASSIGNOR_NAME;
+}
+
+/**
+ * Perform the group assignment given the current members and
+ * topic metadata.
+ *
+ * @param assignmentSpecThe member assignment spec.
+ * @param subscribedTopicDescriber  The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+ * @return The new assignment for the group.
+ */
+@Override
+public GroupAssignment assign(
+AssignmentSpec assignmentSpec,
+SubscribedTopicDescriber subscribedTopicDescriber
+) throws PartitionAssignorException {
+AbstractAssignmentBuilder assignmentBuilder;
+if (assignmentSpec.members().isEmpty())
+return new GroupAssignment(Collections.emptyMap());
+
+if (allSubscriptionsEqual(assignmentSpec.members())) {
+log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
++ "optimized assignment algorithm");
+assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+} else {
+assignmentBuilder = new GeneralUniformAssignmentBuilder();
+log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
++ "general assignment algorithm");
+}
+
+return assignmentBuilder.buildAssignment();
+}
+
+/**
+ * Determines if all members are subscribed to the same list of topic IDs.
+ *
+ * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+ *Assumes the map is non-empty.
+ * @return true if all members have the same subscription list of topic 
IDs,
+ * false otherwise.
+ */
+private boolean allSubscriptionsEqual(Map 
members) {
+Set firstSubscriptionSet = new 
HashSet<>(members.values().iterator().next().subscribedTopicIds());
+for (AssignmentMemberSpec memberSpec : members.values()) {
+Set currentSubscriptionSet = new 
HashSet<>(memberSpec.subscribedTopicIds());
+if (!firstSubscriptionSet.equals(currentSubscriptionSet)) {
+return false;
+}
+}
+return true;
+}
+
+/**
+ * The assignment builder is used to construct the final assignment in a 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-06 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1316806195


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *  balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final Set subscriptionIds;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionIds = new 
HashSet<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+this.rackInfo = new RackInfo(assignmentSpec, subscribedTopicDescriber, 
subscriptionIds);
+this.potentiallyUnfilledMembers = new HashMap<>();
+this.unfilledMembers = new HashMap<>();
+this.newAssignment = new HashMap<>();
+this.useRackAwareStrategy = rackInfo.useRackStrategy;
+// Without rack-aware strategy, tracking current owners of unassigned 
partitions is unnecessary
+// as all sticky partitions are retained until a member meets its 
quota.
+this.currentPartitionOwners = useRackAwareStrategy ? new HashMap<>() : 
Collections.emptyMap();
+}
+
+/**
+ * Here's the step-by-step breakdown of the assignment process:
+ *
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its owner, track it for 
future use.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-06 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1316793219


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##
@@ -0,0 +1,288 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * 
+ * 
+ *  Optimized Uniform Assignment Builder:  This strategy is 
used when all members have subscribed
+ * to the same set of topics.
+ * 
+ * 
+ *  General Uniform Assignment Builder:  This strategy is used 
when members have varied topic
+ * subscriptions.
+ * 
+ * 
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+@Override
+public String name() {
+return UNIFORM_ASSIGNOR_NAME;
+}
+
+/**
+ * Perform the group assignment given the current members and
+ * topic metadata.
+ *
+ * @param assignmentSpecThe member assignment spec.
+ * @param subscribedTopicDescriber  The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+ * @return The new assignment for the group.
+ */
+@Override
+public GroupAssignment assign(
+AssignmentSpec assignmentSpec,
+SubscribedTopicDescriber subscribedTopicDescriber
+) throws PartitionAssignorException {
+AbstractAssignmentBuilder assignmentBuilder;
+if (assignmentSpec.members().isEmpty())
+return new GroupAssignment(Collections.emptyMap());
+
+if (allSubscriptionsEqual(assignmentSpec.members())) {
+log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
++ "optimized assignment algorithm");
+assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+} else {
+assignmentBuilder = new GeneralUniformAssignmentBuilder();
+log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
++ "general assignment algorithm");
+}
+
+return assignmentBuilder.buildAssignment();
+}
+
+/**
+ * Determines if all members are subscribed to the same list of topic IDs.
+ *
+ * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+ *Assumes the map is non-empty.
+ * @return true if all members have the same subscription list of topic 
IDs,
+ * false otherwise.
+ */
+private boolean allSubscriptionsEqual(Map 
members) {
+Set firstSubscriptionSet = new 
HashSet<>(members.values().iterator().next().subscribedTopicIds());
+for (AssignmentMemberSpec memberSpec : members.values()) {
+Set currentSubscriptionSet = new 
HashSet<>(memberSpec.subscribedTopicIds());
+if (!firstSubscriptionSet.equals(currentSubscriptionSet)) {
+return false;
+}
+}
+return true;
+}
+
+/**
+ * The assignment builder is used to construct the final assignment in a 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-06 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1316792230


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##
@@ -0,0 +1,288 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * 
+ * 
+ *  Optimized Uniform Assignment Builder:  This strategy is 
used when all members have subscribed
+ * to the same set of topics.
+ * 
+ * 
+ *  General Uniform Assignment Builder:  This strategy is used 
when members have varied topic
+ * subscriptions.
+ * 
+ * 
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+@Override
+public String name() {
+return UNIFORM_ASSIGNOR_NAME;
+}
+
+/**
+ * Perform the group assignment given the current members and
+ * topic metadata.
+ *
+ * @param assignmentSpecThe member assignment spec.
+ * @param subscribedTopicDescriber  The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+ * @return The new assignment for the group.
+ */
+@Override
+public GroupAssignment assign(
+AssignmentSpec assignmentSpec,
+SubscribedTopicDescriber subscribedTopicDescriber
+) throws PartitionAssignorException {
+AbstractAssignmentBuilder assignmentBuilder;
+if (assignmentSpec.members().isEmpty())
+return new GroupAssignment(Collections.emptyMap());
+
+if (allSubscriptionsEqual(assignmentSpec.members())) {
+log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
++ "optimized assignment algorithm");
+assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+} else {
+assignmentBuilder = new GeneralUniformAssignmentBuilder();
+log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
++ "general assignment algorithm");
+}
+
+return assignmentBuilder.buildAssignment();
+}
+
+/**
+ * Determines if all members are subscribed to the same list of topic IDs.
+ *
+ * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+ *Assumes the map is non-empty.
+ * @return true if all members have the same subscription list of topic 
IDs,
+ * false otherwise.
+ */
+private boolean allSubscriptionsEqual(Map 
members) {
+Set firstSubscriptionSet = new 
HashSet<>(members.values().iterator().next().subscribedTopicIds());
+for (AssignmentMemberSpec memberSpec : members.values()) {
+Set currentSubscriptionSet = new 
HashSet<>(memberSpec.subscribedTopicIds());
+if (!firstSubscriptionSet.equals(currentSubscriptionSet)) {
+return false;
+}
+}
+return true;
+}
+
+/**
+ * The assignment builder is used to construct the final assignment in a 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-05 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1316456003


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *  balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final Set subscriptionIds;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionIds = new 
HashSet<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+this.rackInfo = new RackInfo(assignmentSpec, subscribedTopicDescriber, 
subscriptionIds);
+this.potentiallyUnfilledMembers = new HashMap<>();
+this.unfilledMembers = new HashMap<>();
+this.newAssignment = new HashMap<>();
+this.useRackAwareStrategy = rackInfo.useRackStrategy;
+// Without rack-aware strategy, tracking current owners of unassigned 
partitions is unnecessary
+// as all sticky partitions are retained until a member meets its 
quota.
+this.currentPartitionOwners = useRackAwareStrategy ? new HashMap<>() : 
Collections.emptyMap();
+}
+
+/**
+ * Here's the step-by-step breakdown of the assignment process:
+ *
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its owner, track it for 
future use.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-05 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1316439651


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *  balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final Set subscriptionIds;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionIds = new 
HashSet<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+this.rackInfo = new RackInfo(assignmentSpec, subscribedTopicDescriber, 
subscriptionIds);
+this.potentiallyUnfilledMembers = new HashMap<>();
+this.unfilledMembers = new HashMap<>();
+this.newAssignment = new HashMap<>();
+this.useRackAwareStrategy = rackInfo.useRackStrategy;

Review Comment:
   yep makes sense I removed it, I thought in the future if we wanted to have a 
flag in the assignor to switch it on or off



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-05 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1316429235


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *  balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;

Review Comment:
   done



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-05 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1316408646


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *  balance > rack matching (when applicable) > stickiness.
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final Set subscriptionIds;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionIds = new 
HashSet<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());

Review Comment:
   yeah correct I added a check in the assign method so its not possible anymore



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-05 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1316406386


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GeneralUniformAssignmentBuilder.java:
##
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class GeneralUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(GeneralUniformAssignmentBuilder.class);
+
+@Override
+protected GroupAssignment buildAssignment() {
+return null;
+

Review Comment:
   ack will remove it but we can ignore this file for now since the 
implementation isn't in yet, just needed the initial template for now to get 
the conditional implementation of the specific assignment builder based on the 
subscriptions.



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-05 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1316396258


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

Review Comment:
   removing this



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-05 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1316390836


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

Review Comment:
   Its not the exact same, that one has topicId mapped to topicPartition here 
its just the partition number



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-05 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1316096433


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

Review Comment:
   Yep I'll raise a different PR for range assignor changes



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-04 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1315178928


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java:
##
@@ -0,0 +1,1070 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.junit.jupiter.api.Test;
+
+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.TreeMap;
+
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static 
org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class OptimizedUniformAssignmentBuilderTest {
+private final UniformAssignor assignor = new UniformAssignor();
+private final Uuid topic1Uuid = Uuid.fromString("T1-A4s3VTwiI5CTbEp6POw");
+private final Uuid topic2Uuid = Uuid.fromString("T2-B4s3VTwiI5YHbPp6YUe");
+private final Uuid topic3Uuid = Uuid.fromString("T3-CU8fVTLCz5YMkLoDQsa");
+private final String topic1Name = "topic1";
+private final String topic2Name = "topic2";
+private final String topic3Name = "topic3";
+private final String memberA = "A";
+private final String memberB = "B";
+private final String memberC = "C";
+
+@Test
+public void testOneMemberNoTopicSubscription() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.emptyList(),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void testOneMemberSubscribedToNonexistentTopic() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.singletonList(topic2Uuid),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void 
testFirstAssignmentTwoMembersSubscribedToTwoTopicsNoMemberRacks() {
+Map topicMetadata = new HashMap<>();
+topicMetadata.put(topic1Uuid, new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+));
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-04 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1315178781


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * 
+ * 
+ *  Optimized Uniform Assignment Builder:  This strategy is 
used when all members have subscribed
+ * to the same set of topics.
+ * 
+ * 
+ *  General Uniform Assignment Builder:  This strategy is used 
when members have varied topic
+ * subscriptions.
+ * 
+ * 
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+@Override
+public String name() {
+return UNIFORM_ASSIGNOR_NAME;
+}
+
+/**
+ * Perform the group assignment given the current members and
+ * topic metadata.
+ *
+ * @param assignmentSpecThe member assignment spec.
+ * @param subscribedTopicDescriber  The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+ * @return The new assignment for the group.
+ */
+@Override
+public GroupAssignment assign(
+AssignmentSpec assignmentSpec,
+SubscribedTopicDescriber subscribedTopicDescriber
+) throws PartitionAssignorException {
+
+AbstractAssignmentBuilder assignmentBuilder;
+if (allSubscriptionsEqual(assignmentSpec.members())) {
+log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
++ "optimized assignment algorithm");
+assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+} else {
+assignmentBuilder = new GeneralUniformAssignmentBuilder();
+log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
++ "general assignment algorithm");
+}
+return assignmentBuilder.buildAssignment();
+}
+
+/**
+ * Determines if all members are subscribed to the same list of topic IDs.
+ *
+ * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+ *Assumes the map is non-empty.
+ * @return true if all members have the same subscription list of topic 
IDs,
+ * false otherwise.
+ */
+private boolean allSubscriptionsEqual(Map 
members) {
+Set firstSubscriptionSet = new 
HashSet<>(members.values().iterator().next().subscribedTopicIds());

Review Comment:
   added check but I don't think we need to throw an illegal state exception



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-04 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1315178560


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-04 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1315178473


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());

Review Comment:
   as discussed on call, I added a check in the 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-01 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1312625983


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-01 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1312625675


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-01 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1312625110


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-01 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1312624721


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-01 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1312623868


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-01 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1312619017


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * 
+ * 
+ *  Optimized Uniform Assignment Builder:  This strategy is 
used when all members have subscribed
+ * to the same set of topics.
+ * 
+ * 
+ *  General Uniform Assignment Builder:  This strategy is used 
when members have varied topic
+ * subscriptions.
+ * 
+ * 
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+@Override
+public String name() {
+return UNIFORM_ASSIGNOR_NAME;
+}
+
+/**
+ * Perform the group assignment given the current members and
+ * topic metadata.
+ *
+ * @param assignmentSpecThe member assignment spec.
+ * @param subscribedTopicDescriber  The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+ * @return The new assignment for the group.
+ */
+@Override
+public GroupAssignment assign(
+AssignmentSpec assignmentSpec,
+SubscribedTopicDescriber subscribedTopicDescriber
+) throws PartitionAssignorException {
+
+AbstractAssignmentBuilder assignmentBuilder;
+if (allSubscriptionsEqual(assignmentSpec.members())) {
+log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
++ "optimized assignment algorithm");
+assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+} else {
+assignmentBuilder = new GeneralUniformAssignmentBuilder();
+log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
++ "general assignment algorithm");
+}
+return assignmentBuilder.buildAssignment();
+}
+
+/**
+ * Determines if all members are subscribed to the same list of topic IDs.
+ *
+ * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+ *Assumes the map is non-empty.
+ * @return true if all members have the same subscription list of topic 
IDs,
+ * false otherwise.
+ */
+private boolean allSubscriptionsEqual(Map 
members) {
+boolean allSubscriptionsEqual = true;
+Collection firstSubscriptionList = 
members.values().iterator().next().subscribedTopicIds();
+for (AssignmentMemberSpec memberSpec : members.values()) {
+if 
(!firstSubscriptionList.equals(memberSpec.subscribedTopicIds())) {
+allSubscriptionsEqual = false;
+break;
+}
+}
+return allSubscriptionsEqual;
+}
+
+protected static abstract class AbstractAssignmentBuilder {
+protected abstract GroupAssignment buildAssignment();
+
+protected boolean 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-01 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1312613499


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java:
##
@@ -0,0 +1,1070 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.junit.jupiter.api.Test;
+
+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.TreeMap;
+
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static 
org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class OptimizedUniformAssignmentBuilderTest {
+private final UniformAssignor assignor = new UniformAssignor();
+private final Uuid topic1Uuid = Uuid.fromString("T1-A4s3VTwiI5CTbEp6POw");
+private final Uuid topic2Uuid = Uuid.fromString("T2-B4s3VTwiI5YHbPp6YUe");
+private final Uuid topic3Uuid = Uuid.fromString("T3-CU8fVTLCz5YMkLoDQsa");
+private final String topic1Name = "topic1";
+private final String topic2Name = "topic2";
+private final String topic3Name = "topic3";
+private final String memberA = "A";
+private final String memberB = "B";
+private final String memberC = "C";
+
+@Test
+public void testOneMemberNoTopicSubscription() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.emptyList(),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void testOneMemberSubscribedToNonexistentTopic() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.singletonList(topic2Uuid),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void 
testFirstAssignmentTwoMembersSubscribedToTwoTopicsNoMemberRacks() {
+Map topicMetadata = new HashMap<>();
+topicMetadata.put(topic1Uuid, new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+));
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-01 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1312612036


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java:
##
@@ -0,0 +1,1070 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.junit.jupiter.api.Test;
+
+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.TreeMap;
+
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static 
org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class OptimizedUniformAssignmentBuilderTest {
+private final UniformAssignor assignor = new UniformAssignor();
+private final Uuid topic1Uuid = Uuid.fromString("T1-A4s3VTwiI5CTbEp6POw");
+private final Uuid topic2Uuid = Uuid.fromString("T2-B4s3VTwiI5YHbPp6YUe");
+private final Uuid topic3Uuid = Uuid.fromString("T3-CU8fVTLCz5YMkLoDQsa");
+private final String topic1Name = "topic1";
+private final String topic2Name = "topic2";
+private final String topic3Name = "topic3";
+private final String memberA = "A";
+private final String memberB = "B";
+private final String memberC = "C";
+
+@Test
+public void testOneMemberNoTopicSubscription() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.emptyList(),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void testOneMemberSubscribedToNonexistentTopic() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.singletonList(topic2Uuid),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void 
testFirstAssignmentTwoMembersSubscribedToTwoTopicsNoMemberRacks() {
+Map topicMetadata = new HashMap<>();
+topicMetadata.put(topic1Uuid, new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+));
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-01 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1312612036


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java:
##
@@ -0,0 +1,1070 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.junit.jupiter.api.Test;
+
+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.TreeMap;
+
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static 
org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class OptimizedUniformAssignmentBuilderTest {
+private final UniformAssignor assignor = new UniformAssignor();
+private final Uuid topic1Uuid = Uuid.fromString("T1-A4s3VTwiI5CTbEp6POw");
+private final Uuid topic2Uuid = Uuid.fromString("T2-B4s3VTwiI5YHbPp6YUe");
+private final Uuid topic3Uuid = Uuid.fromString("T3-CU8fVTLCz5YMkLoDQsa");
+private final String topic1Name = "topic1";
+private final String topic2Name = "topic2";
+private final String topic3Name = "topic3";
+private final String memberA = "A";
+private final String memberB = "B";
+private final String memberC = "C";
+
+@Test
+public void testOneMemberNoTopicSubscription() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.emptyList(),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void testOneMemberSubscribedToNonexistentTopic() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.singletonList(topic2Uuid),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void 
testFirstAssignmentTwoMembersSubscribedToTwoTopicsNoMemberRacks() {
+Map topicMetadata = new HashMap<>();
+topicMetadata.put(topic1Uuid, new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+));
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-01 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1312610747


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-09-01 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1312609650


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java:
##
@@ -0,0 +1,1070 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.junit.jupiter.api.Test;
+
+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.TreeMap;
+
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static 
org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class OptimizedUniformAssignmentBuilderTest {
+private final UniformAssignor assignor = new UniformAssignor();
+private final Uuid topic1Uuid = Uuid.fromString("T1-A4s3VTwiI5CTbEp6POw");
+private final Uuid topic2Uuid = Uuid.fromString("T2-B4s3VTwiI5YHbPp6YUe");
+private final Uuid topic3Uuid = Uuid.fromString("T3-CU8fVTLCz5YMkLoDQsa");
+private final String topic1Name = "topic1";
+private final String topic2Name = "topic2";
+private final String topic3Name = "topic3";
+private final String memberA = "A";
+private final String memberB = "B";
+private final String memberC = "C";
+
+@Test
+public void testOneMemberNoTopicSubscription() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.emptyList(),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void testOneMemberSubscribedToNonexistentTopic() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.singletonList(topic2Uuid),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void 
testFirstAssignmentTwoMembersSubscribedToTwoTopicsNoMemberRacks() {
+Map topicMetadata = new HashMap<>();
+topicMetadata.put(topic1Uuid, new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+));
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-31 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1312392605


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java:
##
@@ -0,0 +1,1070 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.junit.jupiter.api.Test;
+
+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.TreeMap;
+
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static 
org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class OptimizedUniformAssignmentBuilderTest {
+private final UniformAssignor assignor = new UniformAssignor();
+private final Uuid topic1Uuid = Uuid.fromString("T1-A4s3VTwiI5CTbEp6POw");
+private final Uuid topic2Uuid = Uuid.fromString("T2-B4s3VTwiI5YHbPp6YUe");
+private final Uuid topic3Uuid = Uuid.fromString("T3-CU8fVTLCz5YMkLoDQsa");
+private final String topic1Name = "topic1";
+private final String topic2Name = "topic2";
+private final String topic3Name = "topic3";
+private final String memberA = "A";
+private final String memberB = "B";
+private final String memberC = "C";
+
+@Test
+public void testOneMemberNoTopicSubscription() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.emptyList(),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void testOneMemberSubscribedToNonexistentTopic() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.singletonList(topic2Uuid),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void 
testFirstAssignmentTwoMembersSubscribedToTwoTopicsNoMemberRacks() {
+Map topicMetadata = new HashMap<>();
+topicMetadata.put(topic1Uuid, new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+));
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-31 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1312392493


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java:
##
@@ -0,0 +1,1070 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.junit.jupiter.api.Test;
+
+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.TreeMap;
+
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static 
org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class OptimizedUniformAssignmentBuilderTest {
+private final UniformAssignor assignor = new UniformAssignor();
+private final Uuid topic1Uuid = Uuid.fromString("T1-A4s3VTwiI5CTbEp6POw");
+private final Uuid topic2Uuid = Uuid.fromString("T2-B4s3VTwiI5YHbPp6YUe");
+private final Uuid topic3Uuid = Uuid.fromString("T3-CU8fVTLCz5YMkLoDQsa");
+private final String topic1Name = "topic1";
+private final String topic2Name = "topic2";
+private final String topic3Name = "topic3";
+private final String memberA = "A";
+private final String memberB = "B";
+private final String memberC = "C";
+
+@Test
+public void testOneMemberNoTopicSubscription() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.emptyList(),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void testOneMemberSubscribedToNonexistentTopic() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.singletonList(topic2Uuid),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void 
testFirstAssignmentTwoMembersSubscribedToTwoTopicsNoMemberRacks() {
+Map topicMetadata = new HashMap<>();
+topicMetadata.put(topic1Uuid, new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+));
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-31 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1312392393


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java:
##
@@ -0,0 +1,1070 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.junit.jupiter.api.Test;
+
+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.TreeMap;
+
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static 
org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class OptimizedUniformAssignmentBuilderTest {
+private final UniformAssignor assignor = new UniformAssignor();
+private final Uuid topic1Uuid = Uuid.fromString("T1-A4s3VTwiI5CTbEp6POw");
+private final Uuid topic2Uuid = Uuid.fromString("T2-B4s3VTwiI5YHbPp6YUe");
+private final Uuid topic3Uuid = Uuid.fromString("T3-CU8fVTLCz5YMkLoDQsa");
+private final String topic1Name = "topic1";
+private final String topic2Name = "topic2";
+private final String topic3Name = "topic3";
+private final String memberA = "A";
+private final String memberB = "B";
+private final String memberC = "C";
+
+@Test
+public void testOneMemberNoTopicSubscription() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.emptyList(),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void testOneMemberSubscribedToNonexistentTopic() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.singletonList(topic2Uuid),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void 
testFirstAssignmentTwoMembersSubscribedToTwoTopicsNoMemberRacks() {
+Map topicMetadata = new HashMap<>();
+topicMetadata.put(topic1Uuid, new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+));
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-30 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1310961579


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java:
##
@@ -0,0 +1,1070 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.junit.jupiter.api.Test;
+
+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.TreeMap;
+
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static 
org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class OptimizedUniformAssignmentBuilderTest {
+private final UniformAssignor assignor = new UniformAssignor();
+private final Uuid topic1Uuid = Uuid.fromString("T1-A4s3VTwiI5CTbEp6POw");
+private final Uuid topic2Uuid = Uuid.fromString("T2-B4s3VTwiI5YHbPp6YUe");
+private final Uuid topic3Uuid = Uuid.fromString("T3-CU8fVTLCz5YMkLoDQsa");
+private final String topic1Name = "topic1";
+private final String topic2Name = "topic2";
+private final String topic3Name = "topic3";
+private final String memberA = "A";
+private final String memberB = "B";
+private final String memberC = "C";
+
+@Test
+public void testOneMemberNoTopicSubscription() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.emptyList(),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void testOneMemberSubscribedToNonexistentTopic() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.singletonList(topic2Uuid),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void 
testFirstAssignmentTwoMembersSubscribedToTwoTopicsNoMemberRacks() {
+Map topicMetadata = new HashMap<>();
+topicMetadata.put(topic1Uuid, new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+));
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-30 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1310953106


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java:
##
@@ -0,0 +1,1070 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.junit.jupiter.api.Test;
+
+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.TreeMap;
+
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static 
org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class OptimizedUniformAssignmentBuilderTest {
+private final UniformAssignor assignor = new UniformAssignor();
+private final Uuid topic1Uuid = Uuid.fromString("T1-A4s3VTwiI5CTbEp6POw");
+private final Uuid topic2Uuid = Uuid.fromString("T2-B4s3VTwiI5YHbPp6YUe");
+private final Uuid topic3Uuid = Uuid.fromString("T3-CU8fVTLCz5YMkLoDQsa");
+private final String topic1Name = "topic1";
+private final String topic2Name = "topic2";
+private final String topic3Name = "topic3";
+private final String memberA = "A";
+private final String memberB = "B";
+private final String memberC = "C";
+
+@Test
+public void testOneMemberNoTopicSubscription() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.emptyList(),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void testOneMemberSubscribedToNonexistentTopic() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.singletonList(topic2Uuid),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void 
testFirstAssignmentTwoMembersSubscribedToTwoTopicsNoMemberRacks() {
+Map topicMetadata = new HashMap<>();
+topicMetadata.put(topic1Uuid, new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+));
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-30 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1310953106


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java:
##
@@ -0,0 +1,1070 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.junit.jupiter.api.Test;
+
+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.TreeMap;
+
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static 
org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class OptimizedUniformAssignmentBuilderTest {
+private final UniformAssignor assignor = new UniformAssignor();
+private final Uuid topic1Uuid = Uuid.fromString("T1-A4s3VTwiI5CTbEp6POw");
+private final Uuid topic2Uuid = Uuid.fromString("T2-B4s3VTwiI5YHbPp6YUe");
+private final Uuid topic3Uuid = Uuid.fromString("T3-CU8fVTLCz5YMkLoDQsa");
+private final String topic1Name = "topic1";
+private final String topic2Name = "topic2";
+private final String topic3Name = "topic3";
+private final String memberA = "A";
+private final String memberB = "B";
+private final String memberC = "C";
+
+@Test
+public void testOneMemberNoTopicSubscription() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.emptyList(),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void testOneMemberSubscribedToNonexistentTopic() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.singletonList(topic2Uuid),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void 
testFirstAssignmentTwoMembersSubscribedToTwoTopicsNoMemberRacks() {
+Map topicMetadata = new HashMap<>();
+topicMetadata.put(topic1Uuid, new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+));
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-30 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1310952891


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java:
##
@@ -0,0 +1,1070 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.junit.jupiter.api.Test;
+
+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.TreeMap;
+
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static 
org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class OptimizedUniformAssignmentBuilderTest {
+private final UniformAssignor assignor = new UniformAssignor();
+private final Uuid topic1Uuid = Uuid.fromString("T1-A4s3VTwiI5CTbEp6POw");
+private final Uuid topic2Uuid = Uuid.fromString("T2-B4s3VTwiI5YHbPp6YUe");
+private final Uuid topic3Uuid = Uuid.fromString("T3-CU8fVTLCz5YMkLoDQsa");
+private final String topic1Name = "topic1";
+private final String topic2Name = "topic2";
+private final String topic3Name = "topic3";
+private final String memberA = "A";
+private final String memberB = "B";
+private final String memberC = "C";
+
+@Test
+public void testOneMemberNoTopicSubscription() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.emptyList(),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void testOneMemberSubscribedToNonexistentTopic() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.singletonList(topic2Uuid),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void 
testFirstAssignmentTwoMembersSubscribedToTwoTopicsNoMemberRacks() {
+Map topicMetadata = new HashMap<>();
+topicMetadata.put(topic1Uuid, new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+));
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-30 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1310948792


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java:
##
@@ -0,0 +1,1070 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.junit.jupiter.api.Test;
+
+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.TreeMap;
+
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static 
org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class OptimizedUniformAssignmentBuilderTest {
+private final UniformAssignor assignor = new UniformAssignor();
+private final Uuid topic1Uuid = Uuid.fromString("T1-A4s3VTwiI5CTbEp6POw");
+private final Uuid topic2Uuid = Uuid.fromString("T2-B4s3VTwiI5YHbPp6YUe");
+private final Uuid topic3Uuid = Uuid.fromString("T3-CU8fVTLCz5YMkLoDQsa");
+private final String topic1Name = "topic1";
+private final String topic2Name = "topic2";
+private final String topic3Name = "topic3";
+private final String memberA = "A";
+private final String memberB = "B";
+private final String memberC = "C";
+
+@Test
+public void testOneMemberNoTopicSubscription() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.emptyList(),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void testOneMemberSubscribedToNonexistentTopic() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.singletonList(topic2Uuid),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void 
testFirstAssignmentTwoMembersSubscribedToTwoTopicsNoMemberRacks() {
+Map topicMetadata = new HashMap<>();
+topicMetadata.put(topic1Uuid, new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+));
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-30 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1310941152


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java:
##
@@ -0,0 +1,1070 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.consumer.SubscribedTopicMetadata;
+import org.apache.kafka.coordinator.group.consumer.TopicMetadata;
+import org.junit.jupiter.api.Test;
+
+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.TreeMap;
+
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkAssignment;
+import static 
org.apache.kafka.coordinator.group.AssignmentTestUtil.mkTopicAssignment;
+import static 
org.apache.kafka.coordinator.group.RecordHelpersTest.mkMapOfPartitionRacks;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class OptimizedUniformAssignmentBuilderTest {
+private final UniformAssignor assignor = new UniformAssignor();
+private final Uuid topic1Uuid = Uuid.fromString("T1-A4s3VTwiI5CTbEp6POw");
+private final Uuid topic2Uuid = Uuid.fromString("T2-B4s3VTwiI5YHbPp6YUe");
+private final Uuid topic3Uuid = Uuid.fromString("T3-CU8fVTLCz5YMkLoDQsa");
+private final String topic1Name = "topic1";
+private final String topic2Name = "topic2";
+private final String topic3Name = "topic3";
+private final String memberA = "A";
+private final String memberB = "B";
+private final String memberC = "C";
+
+@Test
+public void testOneMemberNoTopicSubscription() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.emptyList(),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void testOneMemberSubscribedToNonexistentTopic() {
+SubscribedTopicMetadata subscribedTopicMetadata = new 
SubscribedTopicMetadata(
+Collections.singletonMap(
+topic1Uuid,
+new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+)
+)
+);
+
+Map members = Collections.singletonMap(
+memberA,
+new AssignmentMemberSpec(
+Optional.empty(),
+Optional.empty(),
+Collections.singletonList(topic2Uuid),
+Collections.emptyMap()
+)
+);
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);
+
+assertEquals(Collections.emptyMap(), groupAssignment.members());
+}
+
+@Test
+public void 
testFirstAssignmentTwoMembersSubscribedToTwoTopicsNoMemberRacks() {
+Map topicMetadata = new HashMap<>();
+topicMetadata.put(topic1Uuid, new TopicMetadata(
+topic1Uuid,
+topic1Name,
+3,
+mkMapOfPartitionRacks(3)
+));
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-30 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1310930694


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be:
+ *  balance > rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-30 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1310923371


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * 
+ * 
+ *  Optimized Uniform Assignment Builder:  This strategy is 
used when all members have subscribed
+ * to the same set of topics.
+ * 
+ * 
+ *  General Uniform Assignment Builder:  This strategy is used 
when members have varied topic
+ * subscriptions.
+ * 
+ * 
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+@Override
+public String name() {
+return UNIFORM_ASSIGNOR_NAME;
+}
+
+/**
+ * Perform the group assignment given the current members and
+ * topic metadata.
+ *
+ * @param assignmentSpecThe member assignment spec.
+ * @param subscribedTopicDescriber  The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+ * @return The new assignment for the group.
+ */
+@Override
+public GroupAssignment assign(
+AssignmentSpec assignmentSpec,
+SubscribedTopicDescriber subscribedTopicDescriber
+) throws PartitionAssignorException {
+
+AbstractAssignmentBuilder assignmentBuilder;
+if (allSubscriptionsEqual(assignmentSpec.members())) {
+log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
++ "optimized assignment algorithm");
+assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+} else {
+assignmentBuilder = new GeneralUniformAssignmentBuilder();
+log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
++ "general assignment algorithm");
+}
+return assignmentBuilder.buildAssignment();
+}
+
+/**
+ * Determines if all members are subscribed to the same list of topic IDs.
+ *
+ * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+ *Assumes the map is non-empty.
+ * @return true if all members have the same subscription list of topic 
IDs,
+ * false otherwise.
+ */
+private boolean allSubscriptionsEqual(Map 
members) {
+Set firstSubscriptionSet = new 
HashSet<>(members.values().iterator().next().subscribedTopicIds());

Review Comment:
   can members be empty?



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-30 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1310920650


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-30 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1310919719


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-30 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1310906983


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-30 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1310903635


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-30 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1310897847


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-30 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1310892402


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with

Review Comment:
   yeah that's true sorry idr when I changed it, I'll move it
   



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-30 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1310843639


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308181709


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308179490


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308177084


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308174694


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308174183


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308173397


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308173102


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308172074


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308171085


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308169146


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list
+ *  and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetAnExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+this.subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308167658


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.

Review Comment:
   since we mentioned partition's up front in the sentence it makes sense that 
from then onwards "it" always refers to the partition



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308167128


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,390 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with

Review Comment:
   I feel like the code pattern has always been to explain what the assignor 
does at the top instead of at assign/build, and since this is basically the 
main function of the assignor, it shouldn't make a difference if its at the top 
explaining what the Optimized Uniform Assignment Builder does



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308166084


##
reviewers.py:
##
@@ -28,7 +28,7 @@
 def prompt_for_user():
 while True:
 try:
-user_input = input("\nName or email (case insensitive): ") 
+user_input = input("\nName or email (case insensitive): ")

Review Comment:
   this just keeps happening idky, ignore this file we have to force push it in 
the end



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308165924


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java:
##
@@ -103,9 +102,9 @@ public void testOneConsumerSubscribedToNonExistentTopic() {
 );
 
 AssignmentSpec assignmentSpec = new AssignmentSpec(members);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec, 
subscribedTopicMetadata);

Review Comment:
   yes because we decided how we want to handle the non existent topic case in 
this PR and hence this is just a side effect of that



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308165291


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/common/TopicIdPartition.java:
##
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.common;
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.Objects;
+
+public class TopicIdPartition {

Review Comment:
   It's not haha I would've used that otherwise, this has partition as an 
integer whereas that has TopicPartition (another class) as the attribute in 
addition to topic Id



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308164291


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * 
+ * 
+ *  Optimized Uniform Assignment Builder:  This strategy is 
used when all members have subscribed
+ * to the same set of topics.
+ * 
+ * 
+ *  General Uniform Assignment Builder:  This strategy is used 
when members have varied topic
+ * subscriptions.
+ * 
+ * 
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+@Override
+public String name() {
+return UNIFORM_ASSIGNOR_NAME;
+}
+
+/**
+ * Perform the group assignment given the current members and
+ * topic metadata.
+ *
+ * @param assignmentSpecThe member assignment spec.
+ * @param subscribedTopicDescriber  The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+ * @return The new assignment for the group.
+ */
+@Override
+public GroupAssignment assign(
+AssignmentSpec assignmentSpec,
+SubscribedTopicDescriber subscribedTopicDescriber
+) throws PartitionAssignorException {
+
+AbstractAssignmentBuilder assignmentBuilder;
+if (allSubscriptionsEqual(assignmentSpec.members())) {
+log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
++ "optimized assignment algorithm");
+assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+} else {
+assignmentBuilder = new GeneralUniformAssignmentBuilder();
+log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
++ "general assignment algorithm");
+}
+return assignmentBuilder.buildAssignment();
+}
+
+/**
+ * Determines if all members are subscribed to the same list of topic IDs.
+ *
+ * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+ *Assumes the map is non-empty.
+ * @return true if all members have the same subscription list of topic 
IDs,
+ * false otherwise.
+ */
+private boolean allSubscriptionsEqual(Map 
members) {
+boolean allSubscriptionsEqual = true;
+Collection firstSubscriptionList = 
members.values().iterator().next().subscribedTopicIds();
+for (AssignmentMemberSpec memberSpec : members.values()) {
+if 
(!firstSubscriptionList.equals(memberSpec.subscribedTopicIds())) {
+allSubscriptionsEqual = false;
+break;
+}
+}
+return allSubscriptionsEqual;
+}
+
+protected static abstract class AbstractAssignmentBuilder {
+protected abstract GroupAssignment buildAssignment();
+
+protected boolean 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308155008


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * 
+ * 
+ *  Optimized Uniform Assignment Builder:  This strategy is 
used when all members have subscribed
+ * to the same set of topics.
+ * 
+ * 
+ *  General Uniform Assignment Builder:  This strategy is used 
when members have varied topic
+ * subscriptions.
+ * 
+ * 
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+@Override
+public String name() {
+return UNIFORM_ASSIGNOR_NAME;
+}
+
+/**
+ * Perform the group assignment given the current members and
+ * topic metadata.
+ *
+ * @param assignmentSpecThe member assignment spec.
+ * @param subscribedTopicDescriber  The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+ * @return The new assignment for the group.
+ */
+@Override
+public GroupAssignment assign(
+AssignmentSpec assignmentSpec,
+SubscribedTopicDescriber subscribedTopicDescriber
+) throws PartitionAssignorException {
+
+AbstractAssignmentBuilder assignmentBuilder;
+if (allSubscriptionsEqual(assignmentSpec.members())) {
+log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
++ "optimized assignment algorithm");
+assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+} else {
+assignmentBuilder = new GeneralUniformAssignmentBuilder();
+log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
++ "general assignment algorithm");
+}
+return assignmentBuilder.buildAssignment();
+}
+
+/**
+ * Determines if all members are subscribed to the same list of topic IDs.
+ *
+ * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+ *Assumes the map is non-empty.
+ * @return true if all members have the same subscription list of topic 
IDs,
+ * false otherwise.
+ */
+private boolean allSubscriptionsEqual(Map 
members) {
+boolean allSubscriptionsEqual = true;
+Collection firstSubscriptionList = 
members.values().iterator().next().subscribedTopicIds();
+for (AssignmentMemberSpec memberSpec : members.values()) {
+if 
(!firstSubscriptionList.equals(memberSpec.subscribedTopicIds())) {
+allSubscriptionsEqual = false;
+break;
+}
+}
+return allSubscriptionsEqual;
+}
+
+protected static abstract class AbstractAssignmentBuilder {
+protected abstract GroupAssignment buildAssignment();
+
+protected boolean 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308152462


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * 
+ * 
+ *  Optimized Uniform Assignment Builder:  This strategy is 
used when all members have subscribed
+ * to the same set of topics.
+ * 
+ * 
+ *  General Uniform Assignment Builder:  This strategy is used 
when members have varied topic
+ * subscriptions.
+ * 
+ * 
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+@Override
+public String name() {
+return UNIFORM_ASSIGNOR_NAME;
+}
+
+/**
+ * Perform the group assignment given the current members and
+ * topic metadata.
+ *
+ * @param assignmentSpecThe member assignment spec.
+ * @param subscribedTopicDescriber  The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+ * @return The new assignment for the group.
+ */
+@Override
+public GroupAssignment assign(
+AssignmentSpec assignmentSpec,
+SubscribedTopicDescriber subscribedTopicDescriber
+) throws PartitionAssignorException {
+
+AbstractAssignmentBuilder assignmentBuilder;
+if (allSubscriptionsEqual(assignmentSpec.members())) {
+log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
++ "optimized assignment algorithm");
+assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+} else {
+assignmentBuilder = new GeneralUniformAssignmentBuilder();
+log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
++ "general assignment algorithm");
+}
+return assignmentBuilder.buildAssignment();
+}
+
+/**
+ * Determines if all members are subscribed to the same list of topic IDs.
+ *
+ * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+ *Assumes the map is non-empty.
+ * @return true if all members have the same subscription list of topic 
IDs,
+ * false otherwise.
+ */
+private boolean allSubscriptionsEqual(Map 
members) {
+boolean allSubscriptionsEqual = true;
+Collection firstSubscriptionList = 
members.values().iterator().next().subscribedTopicIds();
+for (AssignmentMemberSpec memberSpec : members.values()) {
+if 
(!firstSubscriptionList.equals(memberSpec.subscribedTopicIds())) {
+allSubscriptionsEqual = false;
+break;
+}
+}
+return allSubscriptionsEqual;
+}
+
+protected static abstract class AbstractAssignmentBuilder {
+protected abstract GroupAssignment buildAssignment();
+
+protected boolean 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308151486


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * 
+ * 
+ *  Optimized Uniform Assignment Builder:  This strategy is 
used when all members have subscribed
+ * to the same set of topics.
+ * 
+ * 
+ *  General Uniform Assignment Builder:  This strategy is used 
when members have varied topic
+ * subscriptions.
+ * 
+ * 
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+@Override
+public String name() {
+return UNIFORM_ASSIGNOR_NAME;
+}
+
+/**
+ * Perform the group assignment given the current members and
+ * topic metadata.
+ *
+ * @param assignmentSpecThe member assignment spec.
+ * @param subscribedTopicDescriber  The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+ * @return The new assignment for the group.
+ */
+@Override
+public GroupAssignment assign(
+AssignmentSpec assignmentSpec,
+SubscribedTopicDescriber subscribedTopicDescriber
+) throws PartitionAssignorException {
+
+AbstractAssignmentBuilder assignmentBuilder;
+if (allSubscriptionsEqual(assignmentSpec.members())) {
+log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
++ "optimized assignment algorithm");
+assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+} else {
+assignmentBuilder = new GeneralUniformAssignmentBuilder();
+log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
++ "general assignment algorithm");
+}
+return assignmentBuilder.buildAssignment();
+}
+
+/**
+ * Determines if all members are subscribed to the same list of topic IDs.
+ *
+ * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+ *Assumes the map is non-empty.
+ * @return true if all members have the same subscription list of topic 
IDs,
+ * false otherwise.
+ */
+private boolean allSubscriptionsEqual(Map 
members) {
+boolean allSubscriptionsEqual = true;
+Collection firstSubscriptionList = 
members.values().iterator().next().subscribedTopicIds();
+for (AssignmentMemberSpec memberSpec : members.values()) {
+if 
(!firstSubscriptionList.equals(memberSpec.subscribedTopicIds())) {
+allSubscriptionsEqual = false;
+break;
+}
+}
+return allSubscriptionsEqual;
+}
+
+protected static abstract class AbstractAssignmentBuilder {
+protected abstract GroupAssignment buildAssignment();
+
+protected boolean 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308150974


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * 
+ * 
+ *  Optimized Uniform Assignment Builder:  This strategy is 
used when all members have subscribed
+ * to the same set of topics.
+ * 
+ * 
+ *  General Uniform Assignment Builder:  This strategy is used 
when members have varied topic
+ * subscriptions.
+ * 
+ * 
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+@Override
+public String name() {
+return UNIFORM_ASSIGNOR_NAME;
+}
+
+/**
+ * Perform the group assignment given the current members and
+ * topic metadata.
+ *
+ * @param assignmentSpecThe member assignment spec.
+ * @param subscribedTopicDescriber  The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+ * @return The new assignment for the group.
+ */
+@Override
+public GroupAssignment assign(
+AssignmentSpec assignmentSpec,
+SubscribedTopicDescriber subscribedTopicDescriber
+) throws PartitionAssignorException {
+
+AbstractAssignmentBuilder assignmentBuilder;
+if (allSubscriptionsEqual(assignmentSpec.members())) {
+log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
++ "optimized assignment algorithm");
+assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+} else {
+assignmentBuilder = new GeneralUniformAssignmentBuilder();
+log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
++ "general assignment algorithm");
+}
+return assignmentBuilder.buildAssignment();
+}
+
+/**
+ * Determines if all members are subscribed to the same list of topic IDs.
+ *
+ * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+ *Assumes the map is non-empty.
+ * @return true if all members have the same subscription list of topic 
IDs,
+ * false otherwise.
+ */
+private boolean allSubscriptionsEqual(Map 
members) {
+boolean allSubscriptionsEqual = true;
+Collection firstSubscriptionList = 
members.values().iterator().next().subscribedTopicIds();
+for (AssignmentMemberSpec memberSpec : members.values()) {
+if 
(!firstSubscriptionList.equals(memberSpec.subscribedTopicIds())) {
+allSubscriptionsEqual = false;
+break;
+}
+}
+return allSubscriptionsEqual;
+}
+
+protected static abstract class AbstractAssignmentBuilder {
+protected abstract GroupAssignment buildAssignment();
+
+protected boolean 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308150557


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * 
+ * 
+ *  Optimized Uniform Assignment Builder:  This strategy is 
used when all members have subscribed
+ * to the same set of topics.
+ * 
+ * 
+ *  General Uniform Assignment Builder:  This strategy is used 
when members have varied topic
+ * subscriptions.
+ * 
+ * 
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+@Override
+public String name() {
+return UNIFORM_ASSIGNOR_NAME;
+}
+
+/**
+ * Perform the group assignment given the current members and
+ * topic metadata.
+ *
+ * @param assignmentSpecThe member assignment spec.
+ * @param subscribedTopicDescriber  The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+ * @return The new assignment for the group.
+ */
+@Override
+public GroupAssignment assign(
+AssignmentSpec assignmentSpec,
+SubscribedTopicDescriber subscribedTopicDescriber
+) throws PartitionAssignorException {
+
+AbstractAssignmentBuilder assignmentBuilder;
+if (allSubscriptionsEqual(assignmentSpec.members())) {
+log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
++ "optimized assignment algorithm");
+assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+} else {
+assignmentBuilder = new GeneralUniformAssignmentBuilder();
+log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
++ "general assignment algorithm");
+}
+return assignmentBuilder.buildAssignment();
+}
+
+/**
+ * Determines if all members are subscribed to the same list of topic IDs.
+ *
+ * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+ *Assumes the map is non-empty.
+ * @return true if all members have the same subscription list of topic 
IDs,
+ * false otherwise.
+ */
+private boolean allSubscriptionsEqual(Map 
members) {
+boolean allSubscriptionsEqual = true;
+Collection firstSubscriptionList = 
members.values().iterator().next().subscribedTopicIds();
+for (AssignmentMemberSpec memberSpec : members.values()) {
+if 
(!firstSubscriptionList.equals(memberSpec.subscribedTopicIds())) {
+allSubscriptionsEqual = false;
+break;
+}
+}
+return allSubscriptionsEqual;
+}
+
+protected static abstract class AbstractAssignmentBuilder {
+protected abstract GroupAssignment buildAssignment();
+
+protected boolean 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308143447


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:
##
@@ -0,0 +1,241 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * 
+ * 
+ *  Optimized Uniform Assignment Builder:  This strategy is 
used when all members have subscribed
+ * to the same set of topics.
+ * 
+ * 
+ *  General Uniform Assignment Builder:  This strategy is used 
when members have varied topic
+ * subscriptions.
+ * 
+ * 
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+@Override
+public String name() {
+return UNIFORM_ASSIGNOR_NAME;
+}
+
+/**
+ * Perform the group assignment given the current members and
+ * topic metadata.
+ *
+ * @param assignmentSpecThe member assignment spec.
+ * @param subscribedTopicDescriber  The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+ * @return The new assignment for the group.
+ */
+@Override
+public GroupAssignment assign(
+AssignmentSpec assignmentSpec,
+SubscribedTopicDescriber subscribedTopicDescriber
+) throws PartitionAssignorException {
+
+AbstractAssignmentBuilder assignmentBuilder;
+if (allSubscriptionsEqual(assignmentSpec.members())) {
+log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
++ "optimized assignment algorithm");
+assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+} else {
+assignmentBuilder = new GeneralUniformAssignmentBuilder();
+log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
++ "general assignment algorithm");
+}
+return assignmentBuilder.buildAssignment();
+}
+
+/**
+ * Determines if all members are subscribed to the same list of topic IDs.
+ *
+ * @param members A map of member identifiers to their respective {@code 
AssignmentMemberSpec}.
+ *Assumes the map is non-empty.
+ * @return true if all members have the same subscription list of topic 
IDs,
+ * false otherwise.
+ */
+private boolean allSubscriptionsEqual(Map 
members) {
+boolean allSubscriptionsEqual = true;
+Collection firstSubscriptionList = 
members.values().iterator().next().subscribedTopicIds();
+for (AssignmentMemberSpec memberSpec : members.values()) {
+if 
(!firstSubscriptionList.equals(memberSpec.subscribedTopicIds())) {

Review Comment:
   That's kinda why I used List/Set before but I changed it to Collections on 
receiving comments. But that's a good point, I'll make sure the order doesn't 
matter



##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.java:

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308140708


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java:
##
@@ -17,6 +17,8 @@
 package org.apache.kafka.coordinator.group.assignor;
 
 import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;

Review Comment:
   I feel like a whole PR for minor changes which are kinda related to what I 
learn from doing the other assignors would be unnecessary especially cause even 
minor ones take time to go through the review process and stuff. They might 
just go undone



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



[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-28 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1308137334


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-25 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1305998324


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-25 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1305957286


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-25 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1305956396


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-25 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1305948679


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-25 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1305948679


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-25 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1305945556


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-23 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1303309866


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-23 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1303299811


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-23 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1303273451


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-21 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1300780584


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-21 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1300779466


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-21 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1300637770


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilder.java:
##
@@ -0,0 +1,399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * Assigns Kafka partitions to members of a consumer group ensuring a balanced 
distribution with
+ * considerations for sticky assignments and rack-awareness.
+ * The order of priority of properties during the assignment will be: balance 
> rack matching (when applicable) > stickiness.
+ *
+ *  Here's the step-by-step breakdown of the assignment process:
+ *
+ * 
+ *  Compute the quotas of partitions for each member based on the 
total partitions and member count.
+ *  For existing assignments, retain partitions based on the 
determined quota and member's rack compatibility.
+ *  If a partition's rack mismatches with its member, track it with 
its prior owner.
+ *  Identify members that haven't fulfilled their partition quota or 
are eligible to receive extra partitions.
+ *  Derive the unassigned partitions by taking the difference between 
total partitions and the sticky assignments.
+ *  Depending on members needing extra partitions, select members from 
the potentially unfilled list and add them to the unfilled list.
+ *  Proceed with a round-robin assignment adhering to rack awareness.
+ *  For each unassigned partition, locate the first compatible member 
from the unfilled list.
+ *  If no rack-compatible member is found, revert to the tracked 
current owner.
+ *  If that member can't accommodate the partition due to quota 
limits, resort to a generic round-robin assignment.
+ * 
+ */
+public class OptimizedUniformAssignmentBuilder extends 
UniformAssignor.AbstractAssignmentBuilder {
+private static final Logger log = 
LoggerFactory.getLogger(OptimizedUniformAssignmentBuilder.class);
+private final AssignmentSpec assignmentSpec;
+private final SubscribedTopicDescriber subscribedTopicDescriber;
+// List of topics subscribed to by all members.
+private final List subscriptionList;
+private final RackInfo rackInfo;
+// Count of members to receive an extra partition beyond the minimum quota,
+// to account for the distribution of the remaining partitions.
+private int remainingMembersToGetExtraPartition;
+// Map of members to the remaining number of partitions needed to meet the 
minimum quota,
+// including members eligible for an extra partition.
+private final Map potentiallyUnfilledMembers;
+// Members mapped to the remaining number of partitions needed to meet the 
full quota.
+// Full quota = minQuota + one extra partition (if applicable).
+private Map unfilledMembers;
+private List unassignedPartitions;
+private final Map newAssignment;
+// Tracks the current owner of each partition when using rack-aware 
strategy.
+// Current refers to the existing assignment.
+private final Map currentPartitionOwners;
+// Indicates if a rack aware assignment can be done.
+// True if racks are defined for both members and partitions.
+boolean useRackAwareStrategy;
+
+OptimizedUniformAssignmentBuilder(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) {
+this.assignmentSpec = assignmentSpec;
+this.subscribedTopicDescriber = subscribedTopicDescriber;
+subscriptionList = new 
ArrayList<>(assignmentSpec.members().values().iterator().next().subscribedTopicIds());
+
+RackInfo rackInfo = new RackInfo(assignmentSpec, 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #14182: KAFKA 14515: Optimized Uniform Rack Aware Assignor

2023-08-21 Thread via GitHub


rreddy-22 commented on code in PR #14182:
URL: https://github.com/apache/kafka/pull/14182#discussion_r1300633833


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/UniformAssignor.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.coordinator.group.assignor;
+
+import org.apache.kafka.coordinator.group.common.TopicIdPartition;
+import org.apache.kafka.common.Uuid;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+/**
+ * The Uniform Assignor distributes Kafka topic partitions among group members 
for balanced assignment.
+ * The assignor employs two different strategies based on the nature of topic
+ * subscriptions across the group members:
+ * 
+ * 
+ *  Optimized Uniform Assignment Builder:  This strategy is 
used when all members have subscribed
+ * to the same set of topics.
+ * 
+ * 
+ *  General Uniform Assignment Builder:  This strategy is used 
when members have varied topic
+ * subscriptions.
+ * 
+ * 
+ *
+ * The appropriate strategy is automatically chosen based on the current 
members' topic subscriptions.
+ *
+ * @see OptimizedUniformAssignmentBuilder
+ * @see GeneralUniformAssignmentBuilder
+ */
+public class UniformAssignor implements PartitionAssignor {
+private static final Logger log = 
LoggerFactory.getLogger(UniformAssignor.class);
+public static final String UNIFORM_ASSIGNOR_NAME = "uniform";
+
+@Override
+public String name() {
+return UNIFORM_ASSIGNOR_NAME;
+}
+
+/**
+ * Perform the group assignment given the current members and
+ * topic metadata.
+ *
+ * @param assignmentSpecThe member assignment spec.
+ * @param subscribedTopicDescriber  The topic and cluster metadata 
describer {@link SubscribedTopicDescriber}.
+ * @return The new assignment for the group.
+ */
+@Override
+public GroupAssignment assign(AssignmentSpec assignmentSpec, 
SubscribedTopicDescriber subscribedTopicDescriber) throws 
PartitionAssignorException {
+AbstractAssignmentBuilder assignmentBuilder;
+if (allSubscriptionsEqual(assignmentSpec.members())) {
+log.debug("Detected that all members are subscribed to the same 
set of topics, invoking the "
++ "optimized assignment algorithm");
+assignmentBuilder = new 
OptimizedUniformAssignmentBuilder(assignmentSpec, subscribedTopicDescriber);
+} else {
+assignmentBuilder = new GeneralUniformAssignmentBuilder();
+log.debug("Detected that all members are subscribed to a different 
set of topics, invoking the "
++ "general assignment algorithm");
+}
+return assignmentBuilder.buildAssignment();
+}
+
+private boolean allSubscriptionsEqual(Map 
members) {
+boolean areAllSubscriptionsEqual = true;
+Collection firstSubscriptionList = 
members.values().iterator().next().subscribedTopicIds();
+for (AssignmentMemberSpec memberSpec : members.values()) {
+if 
(!firstSubscriptionList.equals(memberSpec.subscribedTopicIds())) {
+areAllSubscriptionsEqual = false;
+break;
+}
+}
+return areAllSubscriptionsEqual;
+}
+
+protected static abstract class AbstractAssignmentBuilder {
+protected abstract GroupAssignment buildAssignment();
+
+protected boolean useRackAwareAssignment(Set consumerRacks, 
Set partitionRacks, Map> 
racksPerPartition) {
+if (consumerRacks.isEmpty() || Collections.disjoint(consumerRacks, 
partitionRacks))
+return false;
+else {
+return 
!racksPerPartition.values().stream().allMatch(partitionRacks::equals);
+}
+}
+
+protected List 

  1   2   >