[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-19 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java:
##
@@ -0,0 +1,240 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are as follows:
+ * 
+ *  Each member must get at least one partition for every topic that it is 
subscribed to. The only exception is when
+ *  the number of subscribed members is greater than the number of 
partitions for that topic. (Range) 
+ *  Partitions should be assigned to members in a way that facilitates the 
join operation when required. (Range) 
+ *This can only be done if every member is subscribed to the same topics 
and the topics are co-partitioned.
+ *Two streams are co-partitioned if the following conditions are met:
+ *
+ *  The keys must have the same schemas.
+ *  The topics involved must have the same number of partitions.
+ *
+ *  Members should retain as much as their previous assignment as possible 
to reduce the number of partition movements during reassignment. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm includes the following steps:
+ * 
+ *  Generate a map of membersPerTopic using the given member 
subscriptions.
+ *  Generate a list of members (potentiallyUnfilledMembers) 
that have not met the minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  Add members from the potentiallyUnfilled list to the 
Unfilled list if they haven't met the total required quota i.e. 
minimum number of partitions per member + 1 (if member is designated to receive 
one of the excess partitions) 
+ *  Generate a list of unassigned partitions by calculating the difference 
between total partitions and already assigned (sticky) partitions 
+ *  Iterate through unfilled members and assign partitions from the 
unassigned partitions 
+ * 
+ * 
+ *
+ */
+public class RangeAssignor implements PartitionAssignor {
+
+private static final Logger log = 
LoggerFactory.getLogger(RangeAssignor.class);
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+static class RemainingAssignmentsForMember {
+private final String memberId;
+private final Integer remaining;
+
+public RemainingAssignmentsForMember(String memberId, Integer 
remaining) {
+this.memberId = memberId;
+this.remaining = remaining;
+}
+
+public String memberId() {
+return memberId;
+}
+
+public Integer remaining() {
+return remaining;
+}
+
+}
+
+private Map> membersPerTopic(final AssignmentSpec 
assignmentSpec) {
+Map> membersPerTopic = new HashMap<>();
+Map membersData = 
assignmentSpec.members();
+
+membersData.forEach((memberId, memberMetadata) -> {
+Collection topics = memberMetadata.subscribedTopicIds();
+for (Uuid topicId: topics) {
+// Only topics that are present in both the subscribed topics 
list and the topic metadata should be considered for assignment.
+if (assignmentSpec.topics().containsKey(topicId)) {
+membersPerTopic.computeIfAbsent(topicId, k -> new 
ArrayList<>()).add(memberId);
+} else {
+log.info(memberId + " subscribed to topic " + topicId + " 
which doesn't exist in the topic metadata");
+}
+}
+});
+
+return membersPerTopic;
+}
+
+private 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-19 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java:
##
@@ -0,0 +1,240 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are as follows:
+ * 
+ *  Each member must get at least one partition for every topic that it is 
subscribed to. The only exception is when
+ *  the number of subscribed members is greater than the number of 
partitions for that topic. (Range) 
+ *  Partitions should be assigned to members in a way that facilitates the 
join operation when required. (Range) 
+ *This can only be done if every member is subscribed to the same topics 
and the topics are co-partitioned.
+ *Two streams are co-partitioned if the following conditions are met:
+ *
+ *  The keys must have the same schemas.
+ *  The topics involved must have the same number of partitions.
+ *
+ *  Members should retain as much as their previous assignment as possible 
to reduce the number of partition movements during reassignment. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm includes the following steps:
+ * 
+ *  Generate a map of membersPerTopic using the given member 
subscriptions.
+ *  Generate a list of members (potentiallyUnfilledMembers) 
that have not met the minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  Add members from the potentiallyUnfilled list to the 
Unfilled list if they haven't met the total required quota i.e. 
minimum number of partitions per member + 1 (if member is designated to receive 
one of the excess partitions) 
+ *  Generate a list of unassigned partitions by calculating the difference 
between total partitions and already assigned (sticky) partitions 
+ *  Iterate through unfilled members and assign partitions from the 
unassigned partitions 
+ * 
+ * 
+ *
+ */
+public class RangeAssignor implements PartitionAssignor {
+
+private static final Logger log = 
LoggerFactory.getLogger(RangeAssignor.class);
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+static class RemainingAssignmentsForMember {
+private final String memberId;
+private final Integer remaining;
+
+public RemainingAssignmentsForMember(String memberId, Integer 
remaining) {
+this.memberId = memberId;
+this.remaining = remaining;
+}
+
+public String memberId() {
+return memberId;
+}
+
+public Integer remaining() {
+return remaining;
+}
+
+}
+
+private Map> membersPerTopic(final AssignmentSpec 
assignmentSpec) {
+Map> membersPerTopic = new HashMap<>();
+Map membersData = 
assignmentSpec.members();
+
+membersData.forEach((memberId, memberMetadata) -> {
+Collection topics = memberMetadata.subscribedTopicIds();
+for (Uuid topicId: topics) {
+// Only topics that are present in both the subscribed topics 
list and the topic metadata should be considered for assignment.
+if (assignmentSpec.topics().containsKey(topicId)) {
+membersPerTopic.computeIfAbsent(topicId, k -> new 
ArrayList<>()).add(memberId);
+} else {
+log.info(memberId + " subscribed to topic " + topicId + " 
which doesn't exist in the topic metadata");
+}
+}
+});
+
+return membersPerTopic;
+}
+
+private 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-19 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java:
##
@@ -0,0 +1,240 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are as follows:
+ * 
+ *  Each member must get at least one partition for every topic that it is 
subscribed to. The only exception is when
+ *  the number of subscribed members is greater than the number of 
partitions for that topic. (Range) 
+ *  Partitions should be assigned to members in a way that facilitates the 
join operation when required. (Range) 
+ *This can only be done if every member is subscribed to the same topics 
and the topics are co-partitioned.
+ *Two streams are co-partitioned if the following conditions are met:
+ *
+ *  The keys must have the same schemas.
+ *  The topics involved must have the same number of partitions.
+ *
+ *  Members should retain as much as their previous assignment as possible 
to reduce the number of partition movements during reassignment. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm includes the following steps:
+ * 
+ *  Generate a map of membersPerTopic using the given member 
subscriptions.
+ *  Generate a list of members (potentiallyUnfilledMembers) 
that have not met the minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  Add members from the potentiallyUnfilled list to the 
Unfilled list if they haven't met the total required quota i.e. 
minimum number of partitions per member + 1 (if member is designated to receive 
one of the excess partitions) 
+ *  Generate a list of unassigned partitions by calculating the difference 
between total partitions and already assigned (sticky) partitions 
+ *  Iterate through unfilled members and assign partitions from the 
unassigned partitions 
+ * 
+ * 
+ *
+ */
+public class RangeAssignor implements PartitionAssignor {
+
+private static final Logger log = 
LoggerFactory.getLogger(RangeAssignor.class);
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+static class RemainingAssignmentsForMember {
+private final String memberId;
+private final Integer remaining;
+
+public RemainingAssignmentsForMember(String memberId, Integer 
remaining) {
+this.memberId = memberId;
+this.remaining = remaining;
+}
+
+public String memberId() {
+return memberId;
+}
+
+public Integer remaining() {
+return remaining;
+}
+
+}
+
+private Map> membersPerTopic(final AssignmentSpec 
assignmentSpec) {
+Map> membersPerTopic = new HashMap<>();
+Map membersData = 
assignmentSpec.members();
+
+membersData.forEach((memberId, memberMetadata) -> {
+Collection topics = memberMetadata.subscribedTopicIds();
+for (Uuid topicId: topics) {
+// Only topics that are present in both the subscribed topics 
list and the topic metadata should be considered for assignment.
+if (assignmentSpec.topics().containsKey(topicId)) {
+membersPerTopic.computeIfAbsent(topicId, k -> new 
ArrayList<>()).add(memberId);
+} else {
+log.info(memberId + " subscribed to topic " + topicId + " 
which doesn't exist in the topic metadata");
+}
+}
+});
+
+return membersPerTopic;
+}
+
+private 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-19 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java:
##
@@ -0,0 +1,240 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.

Review Comment:
   the formatting is really off without them, that's why we had to add them



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-19 Thread via GitHub


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


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java:
##
@@ -0,0 +1,416 @@
+/*
+ * 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.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 static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class RangeAssignorTest {
+private final RangeAssignor assignor = new RangeAssignor();
+private final Uuid topic1Uuid = Uuid.randomUuid();
+private final Uuid topic2Uuid = Uuid.randomUuid();
+private final Uuid topic3Uuid = Uuid.randomUuid();
+private final String consumerA = "A";
+private final String consumerB = "B";
+private final String consumerC = "C";
+
+@Test
+public void testOneConsumerNoTopic() {
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(3));
+Map members = new HashMap<>();
+List subscribedTopics = new ArrayList<>();
+members.computeIfAbsent(consumerA, k -> new 
AssignmentMemberSpec(Optional.empty(), Optional.empty(), subscribedTopics, new 
HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec);
+
+assertTrue(groupAssignment.members().isEmpty());
+}
+
+@Test
+public void testOneConsumerNonExistentTopic() {
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(3));
+Map members = new HashMap<>();
+List subscribedTopics = new ArrayList<>();
+subscribedTopics.add(topic2Uuid);
+members.computeIfAbsent(consumerA, k -> new 
AssignmentMemberSpec(Optional.empty(), Optional.empty(), subscribedTopics, new 
HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec);
+
+assertTrue(groupAssignment.members().isEmpty());
+}
+
+@Test
+public void testFirstAssignmentTwoConsumersTwoTopicsSameSubscriptions() {
+// A -> T1, T3 // B -> T1, T3 // T1 -> 3 Partitions // T3 -> 2 
Partitions
+// Topics
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(3));
+topics.put(topic3Uuid, new AssignmentTopicMetadata(2));
+// Members
+Map members = new HashMap<>();
+// Consumer A
+List subscribedTopicsA = new 
ArrayList<>(Arrays.asList(topic1Uuid, topic3Uuid));

Review Comment:
   you're right, I've removed them!



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-18 Thread via GitHub


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


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java:
##
@@ -0,0 +1,416 @@
+/*
+ * 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.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 static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class RangeAssignorTest {
+private final RangeAssignor assignor = new RangeAssignor();
+private final Uuid topic1Uuid = Uuid.randomUuid();
+private final Uuid topic2Uuid = Uuid.randomUuid();
+private final Uuid topic3Uuid = Uuid.randomUuid();
+private final String consumerA = "A";
+private final String consumerB = "B";
+private final String consumerC = "C";
+
+@Test
+public void testOneConsumerNoTopic() {
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(3));
+Map members = new HashMap<>();
+List subscribedTopics = new ArrayList<>();
+members.computeIfAbsent(consumerA, k -> new 
AssignmentMemberSpec(Optional.empty(), Optional.empty(), subscribedTopics, new 
HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec);
+
+assertTrue(groupAssignment.members().isEmpty());
+}
+
+@Test
+public void testOneConsumerNonExistentTopic() {
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(3));
+Map members = new HashMap<>();
+List subscribedTopics = new ArrayList<>();
+subscribedTopics.add(topic2Uuid);
+members.computeIfAbsent(consumerA, k -> new 
AssignmentMemberSpec(Optional.empty(), Optional.empty(), subscribedTopics, new 
HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec);
+
+assertTrue(groupAssignment.members().isEmpty());
+}
+
+@Test
+public void testFirstAssignmentTwoConsumersTwoTopicsSameSubscriptions() {
+// A -> T1, T3 // B -> T1, T3 // T1 -> 3 Partitions // T3 -> 2 
Partitions
+// Topics
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(3));
+topics.put(topic3Uuid, new AssignmentTopicMetadata(2));
+// Members
+Map members = new HashMap<>();
+// Consumer A
+List subscribedTopicsA = new 
ArrayList<>(Arrays.asList(topic1Uuid, topic3Uuid));
+members.put(consumerA, new AssignmentMemberSpec(Optional.empty(), 
Optional.empty(), subscribedTopicsA, new HashMap<>()));
+// Consumer B
+List subscribedTopicsB = new 
ArrayList<>(Arrays.asList(topic1Uuid, topic3Uuid));
+members.put(consumerB, new AssignmentMemberSpec(Optional.empty(), 
Optional.empty(), subscribedTopicsB, new HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment computedAssignment = assignor.assign(assignmentSpec);
+
+Map>> expectedAssignment = new HashMap<>();
+// Topic 1 Partitions Assignment
+expectedAssignment.computeIfAbsent(topic1Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Arrays.asList(0, 1)));
+expectedAssignment.computeIfAbsent(topic1Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(2)));
+// Topic 3 Partitions Assignment
+expectedAssignment.computeIfAbsent(topic3Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(0)));
+expectedAssignment.computeIfAbsent(topic3Uuid, k -> new 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-18 Thread via GitHub


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


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java:
##
@@ -0,0 +1,416 @@
+/*
+ * 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.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 static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class RangeAssignorTest {
+private final RangeAssignor assignor = new RangeAssignor();
+private final Uuid topic1Uuid = Uuid.randomUuid();
+private final Uuid topic2Uuid = Uuid.randomUuid();
+private final Uuid topic3Uuid = Uuid.randomUuid();
+private final String consumerA = "A";
+private final String consumerB = "B";
+private final String consumerC = "C";
+
+@Test
+public void testOneConsumerNoTopic() {
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(3));
+Map members = new HashMap<>();
+List subscribedTopics = new ArrayList<>();
+members.computeIfAbsent(consumerA, k -> new 
AssignmentMemberSpec(Optional.empty(), Optional.empty(), subscribedTopics, new 
HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec);
+
+assertTrue(groupAssignment.members().isEmpty());
+}
+
+@Test
+public void testOneConsumerNonExistentTopic() {
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(3));
+Map members = new HashMap<>();
+List subscribedTopics = new ArrayList<>();
+subscribedTopics.add(topic2Uuid);
+members.computeIfAbsent(consumerA, k -> new 
AssignmentMemberSpec(Optional.empty(), Optional.empty(), subscribedTopics, new 
HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec);
+
+assertTrue(groupAssignment.members().isEmpty());
+}
+
+@Test
+public void testFirstAssignmentTwoConsumersTwoTopicsSameSubscriptions() {
+// A -> T1, T3 // B -> T1, T3 // T1 -> 3 Partitions // T3 -> 2 
Partitions
+// Topics
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(3));
+topics.put(topic3Uuid, new AssignmentTopicMetadata(2));
+// Members
+Map members = new HashMap<>();
+// Consumer A
+List subscribedTopicsA = new 
ArrayList<>(Arrays.asList(topic1Uuid, topic3Uuid));
+members.put(consumerA, new AssignmentMemberSpec(Optional.empty(), 
Optional.empty(), subscribedTopicsA, new HashMap<>()));
+// Consumer B
+List subscribedTopicsB = new 
ArrayList<>(Arrays.asList(topic1Uuid, topic3Uuid));
+members.put(consumerB, new AssignmentMemberSpec(Optional.empty(), 
Optional.empty(), subscribedTopicsB, new HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment computedAssignment = assignor.assign(assignmentSpec);
+
+Map>> expectedAssignment = new HashMap<>();
+// Topic 1 Partitions Assignment
+expectedAssignment.computeIfAbsent(topic1Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Arrays.asList(0, 1)));
+expectedAssignment.computeIfAbsent(topic1Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(2)));
+// Topic 3 Partitions Assignment
+expectedAssignment.computeIfAbsent(topic3Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(0)));
+expectedAssignment.computeIfAbsent(topic3Uuid, k -> new 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-18 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java:
##
@@ -0,0 +1,240 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are as follows:
+ * 
+ *  Each member must get at least one partition for every topic that it is 
subscribed to. The only exception is when
+ *  the number of subscribed members is greater than the number of 
partitions for that topic. (Range) 
+ *  Partitions should be assigned to members in a way that facilitates the 
join operation when required. (Range) 
+ *This can only be done if every member is subscribed to the same topics 
and the topics are co-partitioned.
+ *Two streams are co-partitioned if the following conditions are met:
+ *
+ *  The keys must have the same schemas.
+ *  The topics involved must have the same number of partitions.
+ *
+ *  Members should retain as much as their previous assignment as possible 
to reduce the number of partition movements during reassignment. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm includes the following steps:
+ * 
+ *  Generate a map of membersPerTopic using the given member 
subscriptions.
+ *  Generate a list of members (potentiallyUnfilledMembers) 
that have not met the minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  Add members from the potentiallyUnfilled list to the 
Unfilled list if they haven't met the total required quota i.e. 
minimum number of partitions per member + 1 (if member is designated to receive 
one of the excess partitions) 
+ *  Generate a list of unassigned partitions by calculating the difference 
between total partitions and already assigned (sticky) partitions 
+ *  Iterate through unfilled members and assign partitions from the 
unassigned partitions 
+ * 
+ * 
+ *
+ */
+public class RangeAssignor implements PartitionAssignor {
+
+private static final Logger log = 
LoggerFactory.getLogger(RangeAssignor.class);
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+static class RemainingAssignmentsForMember {
+private final String memberId;
+private final Integer remaining;
+
+public RemainingAssignmentsForMember(String memberId, Integer 
remaining) {
+this.memberId = memberId;
+this.remaining = remaining;
+}
+
+public String memberId() {
+return memberId;
+}
+
+public Integer remaining() {
+return remaining;
+}
+
+}
+
+private Map> membersPerTopic(final AssignmentSpec 
assignmentSpec) {
+Map> membersPerTopic = new HashMap<>();
+Map membersData = 
assignmentSpec.members();
+
+membersData.forEach((memberId, memberMetadata) -> {
+Collection topics = memberMetadata.subscribedTopicIds();
+for (Uuid topicId: topics) {
+// Only topics that are present in both the subscribed topics 
list and the topic metadata should be considered for assignment.
+if (assignmentSpec.topics().containsKey(topicId)) {
+membersPerTopic.computeIfAbsent(topicId, k -> new 
ArrayList<>()).add(memberId);
+} else {
+log.info(memberId + " subscribed to topic " + topicId + " 
which doesn't exist in the topic metadata");
+}
+}
+});
+
+return membersPerTopic;
+}
+
+private 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-18 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java:
##
@@ -0,0 +1,240 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are as follows:
+ * 
+ *  Each member must get at least one partition for every topic that it is 
subscribed to. The only exception is when
+ *  the number of subscribed members is greater than the number of 
partitions for that topic. (Range) 
+ *  Partitions should be assigned to members in a way that facilitates the 
join operation when required. (Range) 
+ *This can only be done if every member is subscribed to the same topics 
and the topics are co-partitioned.
+ *Two streams are co-partitioned if the following conditions are met:
+ *
+ *  The keys must have the same schemas.
+ *  The topics involved must have the same number of partitions.
+ *
+ *  Members should retain as much as their previous assignment as possible 
to reduce the number of partition movements during reassignment. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm includes the following steps:

Review Comment:
   cool
   



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-18 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/RangeAssignor.java:
##
@@ -0,0 +1,240 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.

Review Comment:
   remove the whole thing or just the html tags



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-18 Thread via GitHub


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


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java:
##
@@ -0,0 +1,416 @@
+/*
+ * 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.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 static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class RangeAssignorTest {
+private final RangeAssignor assignor = new RangeAssignor();
+private final Uuid topic1Uuid = Uuid.randomUuid();
+private final Uuid topic2Uuid = Uuid.randomUuid();
+private final Uuid topic3Uuid = Uuid.randomUuid();
+private final String consumerA = "A";
+private final String consumerB = "B";
+private final String consumerC = "C";
+
+@Test
+public void testOneConsumerNoTopic() {
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(3));
+Map members = new HashMap<>();
+List subscribedTopics = new ArrayList<>();
+members.computeIfAbsent(consumerA, k -> new 
AssignmentMemberSpec(Optional.empty(), Optional.empty(), subscribedTopics, new 
HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec);
+
+assertTrue(groupAssignment.members().isEmpty());
+}
+
+@Test
+public void testOneConsumerNonExistentTopic() {
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(3));
+Map members = new HashMap<>();
+List subscribedTopics = new ArrayList<>();
+subscribedTopics.add(topic2Uuid);
+members.computeIfAbsent(consumerA, k -> new 
AssignmentMemberSpec(Optional.empty(), Optional.empty(), subscribedTopics, new 
HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec);
+
+assertTrue(groupAssignment.members().isEmpty());
+}
+
+@Test
+public void testFirstAssignmentTwoConsumersTwoTopicsSameSubscriptions() {
+// A -> T1, T3 // B -> T1, T3 // T1 -> 3 Partitions // T3 -> 2 
Partitions
+// Topics
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(3));
+topics.put(topic3Uuid, new AssignmentTopicMetadata(2));
+// Members
+Map members = new HashMap<>();
+// Consumer A
+List subscribedTopicsA = new 
ArrayList<>(Arrays.asList(topic1Uuid, topic3Uuid));
+members.put(consumerA, new AssignmentMemberSpec(Optional.empty(), 
Optional.empty(), subscribedTopicsA, new HashMap<>()));
+// Consumer B
+List subscribedTopicsB = new 
ArrayList<>(Arrays.asList(topic1Uuid, topic3Uuid));
+members.put(consumerB, new AssignmentMemberSpec(Optional.empty(), 
Optional.empty(), subscribedTopicsB, new HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment computedAssignment = assignor.assign(assignmentSpec);
+
+Map>> expectedAssignment = new HashMap<>();
+// Topic 1 Partitions Assignment
+expectedAssignment.computeIfAbsent(topic1Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Arrays.asList(0, 1)));
+expectedAssignment.computeIfAbsent(topic1Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(2)));
+// Topic 3 Partitions Assignment
+expectedAssignment.computeIfAbsent(topic3Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(0)));
+expectedAssignment.computeIfAbsent(topic3Uuid, k -> new 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+private static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+private static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+Collection topics = memberMetadata.subscribedTopics;
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+private static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+private static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+Collection topics = memberMetadata.subscribedTopics;
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+private static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+private static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+Collection topics = memberMetadata.subscribedTopics;
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+private static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+private static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+Collection topics = memberMetadata.subscribedTopics;
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+private static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+private static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+Collection topics = memberMetadata.subscribedTopics;
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+private static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+private static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+Collection topics = memberMetadata.subscribedTopics;
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+private static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+private static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+Collection topics = memberMetadata.subscribedTopics;
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+private static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+private static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+Collection topics = memberMetadata.subscribedTopics;
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+private static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+private static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+Collection topics = memberMetadata.subscribedTopics;
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {

Review Comment:
   ok



-- 
This is an automated message from the Apache Git Service.
To 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+private static  void putList(Map> map, K key, V value) {

Review Comment:
   removed



##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+private static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+private static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {

Review Comment:
   changed 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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+private static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+private static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)

Review Comment:
   removed the whole comment cause unnecessary



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+private static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+private static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();

Review Comment:
   changed to membersPerTopic



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 

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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 

Review Comment:
   sounds good thanks!



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,257 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static java.lang.Math.min;
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-

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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/GroupAssignment.java:
##
@@ -35,6 +35,10 @@ public GroupAssignment(
 this.members = members;
 }
 
+public Map getMembers() {

Review Comment:
   changed in interface changes PR



##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/MemberAssignment.java:
##
@@ -16,25 +16,28 @@
  */
 package org.apache.kafka.coordinator.group.assignor;
 
-import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.Uuid;
 
-import java.util.Collection;
+import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
+
 
 /**
  * The partition assignment for a consumer group member.
  */
 public class MemberAssignment {
-/**
- * The target partitions assigned to this member.
- */
-final Collection targetPartitions;
-
-public MemberAssignment(
-Collection targetPartitions
-) {
-Objects.requireNonNull(targetPartitions);
-this.targetPartitions = targetPartitions;
+
+private final Map> assignmentPerTopic;
+
+
+public MemberAssignment(Map> assignmentPerTopic) {
+Objects.requireNonNull(assignmentPerTopic);
+this.assignmentPerTopic = assignmentPerTopic;
+}
+
+public Map> getAssignmentPerTopic() {

Review Comment:
   changed in interface changes PR



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/AssignmentMemberSpec.java:
##
@@ -37,29 +40,28 @@ public class AssignmentMemberSpec {
 final Optional rackId;
 
 /**
- * The topics that the member is subscribed to.
+ * The topicIds of topics that the member is subscribed to.
  */
-final Collection subscribedTopics;
+final Collection subscribedTopics;
 
 /**
- * The current target partitions of the member.
+ * Maps the partitions assigned for this member per topicId

Review Comment:
   changed in interface 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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-06 Thread via GitHub


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


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignorTest.java:
##
@@ -0,0 +1,389 @@
+/*
+ * 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.junit.jupiter.api.Test;
+
+import java.util.Map;
+import java.util.Set;
+
+
+
+import java.util.*;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class ServerSideStickyRangeAssignorTest {
+
+private final ServerSideStickyRangeAssignor assignor = new 
ServerSideStickyRangeAssignor();
+
+private final String topic1Name = "topic1";
+private final Uuid topic1Uuid = Uuid.randomUuid();
+
+private final String topic2Name = "topic2";
+private final Uuid topic2Uuid = Uuid.randomUuid();
+
+private final String topic3Name = "topic3";
+private final Uuid topic3Uuid = Uuid.randomUuid();
+
+private final String consumerA = "A";
+private final String consumerB = "B";
+private final String consumerC = "C";
+
+@Test
+public void testOneConsumerNoTopic() {
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(topic1Name, 3));
+Map members = new HashMap<>();
+List subscribedTopics = new ArrayList<>();
+members.computeIfAbsent(consumerA, k -> new 
AssignmentMemberSpec(Optional.empty(), Optional.empty(), subscribedTopics, new 
HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec);
+
+assertTrue(groupAssignment.getMembers().isEmpty());
+}
+
+@Test
+public void testFirstAssignmentTwoConsumersTwoTopicsSameSubscriptions() {
+// A -> T1, T3 // B -> T1, T3 // T1 -> 3 Partitions // T3 -> 2 
Partitions
+// Topics
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(topic1Name, 3));
+topics.put(topic3Uuid, new AssignmentTopicMetadata(topic3Name, 2));
+// Members
+Map members = new HashMap<>();
+// Consumer A
+List subscribedTopicsA = new 
ArrayList<>(Arrays.asList(topic1Uuid, topic3Uuid));
+members.put(consumerA, new AssignmentMemberSpec(Optional.empty(), 
Optional.empty(), subscribedTopicsA, new HashMap<>()));
+// Consumer B
+List subscribedTopicsB = new 
ArrayList<>(Arrays.asList(topic1Uuid, topic3Uuid));
+members.put(consumerB, new AssignmentMemberSpec(Optional.empty(), 
Optional.empty(), subscribedTopicsB, new HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment computedAssignment = assignor.assign(assignmentSpec);
+
+Map>> expectedAssignment = new HashMap<>();
+// Topic 1 Partitions Assignment
+expectedAssignment.computeIfAbsent(topic1Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Arrays.asList(0, 1)));
+expectedAssignment.computeIfAbsent(topic1Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(2)));
+// Topic 3 Partitions Assignment
+expectedAssignment.computeIfAbsent(topic3Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(0)));
+expectedAssignment.computeIfAbsent(topic3Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(1)));
+
+assertAssignment(expectedAssignment, computedAssignment);
+}
+
+@Test
+public void 
testFirstAssignmentThreeConsumersThreeTopicsDifferentSubscriptions() {
+// A -> T1, T2 // B -> T3 // C -> T2, T3 // T1 -> 3 Partitions // T2 
-> 3 Partitions // T3 -> 2 Partitions
+// Topics
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(topic1Name, 3));
+topics.put(topic2Uuid, new AssignmentTopicMetadata(topic1Name, 3));
+topics.put(topic3Uuid, new AssignmentTopicMetadata(topic3Name, 2));
+// Members
+Map members = new HashMap<>();
+// 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-06 Thread via GitHub


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


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignorTest.java:
##
@@ -0,0 +1,389 @@
+/*
+ * 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.junit.jupiter.api.Test;
+
+import java.util.Map;
+import java.util.Set;
+
+
+
+import java.util.*;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class ServerSideStickyRangeAssignorTest {
+
+private final ServerSideStickyRangeAssignor assignor = new 
ServerSideStickyRangeAssignor();
+
+private final String topic1Name = "topic1";
+private final Uuid topic1Uuid = Uuid.randomUuid();
+
+private final String topic2Name = "topic2";
+private final Uuid topic2Uuid = Uuid.randomUuid();
+
+private final String topic3Name = "topic3";
+private final Uuid topic3Uuid = Uuid.randomUuid();
+
+private final String consumerA = "A";
+private final String consumerB = "B";
+private final String consumerC = "C";
+
+@Test
+public void testOneConsumerNoTopic() {
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(topic1Name, 3));
+Map members = new HashMap<>();
+List subscribedTopics = new ArrayList<>();
+members.computeIfAbsent(consumerA, k -> new 
AssignmentMemberSpec(Optional.empty(), Optional.empty(), subscribedTopics, new 
HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec);
+
+assertTrue(groupAssignment.getMembers().isEmpty());
+}
+
+@Test
+public void testFirstAssignmentTwoConsumersTwoTopicsSameSubscriptions() {
+// A -> T1, T3 // B -> T1, T3 // T1 -> 3 Partitions // T3 -> 2 
Partitions
+// Topics
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(topic1Name, 3));
+topics.put(topic3Uuid, new AssignmentTopicMetadata(topic3Name, 2));
+// Members
+Map members = new HashMap<>();
+// Consumer A
+List subscribedTopicsA = new 
ArrayList<>(Arrays.asList(topic1Uuid, topic3Uuid));
+members.put(consumerA, new AssignmentMemberSpec(Optional.empty(), 
Optional.empty(), subscribedTopicsA, new HashMap<>()));
+// Consumer B
+List subscribedTopicsB = new 
ArrayList<>(Arrays.asList(topic1Uuid, topic3Uuid));
+members.put(consumerB, new AssignmentMemberSpec(Optional.empty(), 
Optional.empty(), subscribedTopicsB, new HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment computedAssignment = assignor.assign(assignmentSpec);
+
+Map>> expectedAssignment = new HashMap<>();
+// Topic 1 Partitions Assignment
+expectedAssignment.computeIfAbsent(topic1Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Arrays.asList(0, 1)));
+expectedAssignment.computeIfAbsent(topic1Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(2)));
+// Topic 3 Partitions Assignment
+expectedAssignment.computeIfAbsent(topic3Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(0)));
+expectedAssignment.computeIfAbsent(topic3Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(1)));
+
+assertAssignment(expectedAssignment, computedAssignment);
+}
+
+@Test
+public void 
testFirstAssignmentThreeConsumersThreeTopicsDifferentSubscriptions() {
+// A -> T1, T2 // B -> T3 // C -> T2, T3 // T1 -> 3 Partitions // T2 
-> 3 Partitions // T3 -> 2 Partitions
+// Topics
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(topic1Name, 3));
+topics.put(topic2Uuid, new AssignmentTopicMetadata(topic1Name, 3));

Review Comment:
   ohhh yess thanks!
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-06 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {

Review Comment:
   Yep its a very specific type of pair so I custom made this, we could add it 
if its useful in other places as well



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-03 Thread via GitHub


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


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignorTest.java:
##
@@ -0,0 +1,389 @@
+/*
+ * 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.junit.jupiter.api.Test;
+
+import java.util.Map;
+import java.util.Set;
+
+
+
+import java.util.*;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class ServerSideStickyRangeAssignorTest {
+
+private final ServerSideStickyRangeAssignor assignor = new 
ServerSideStickyRangeAssignor();
+
+private final String topic1Name = "topic1";
+private final Uuid topic1Uuid = Uuid.randomUuid();
+
+private final String topic2Name = "topic2";
+private final Uuid topic2Uuid = Uuid.randomUuid();
+
+private final String topic3Name = "topic3";
+private final Uuid topic3Uuid = Uuid.randomUuid();
+
+private final String consumerA = "A";
+private final String consumerB = "B";
+private final String consumerC = "C";
+
+@Test
+public void testOneConsumerNoTopic() {
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(topic1Name, 3));
+Map members = new HashMap<>();
+List subscribedTopics = new ArrayList<>();
+members.computeIfAbsent(consumerA, k -> new 
AssignmentMemberSpec(Optional.empty(), Optional.empty(), subscribedTopics, new 
HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec);
+
+assertTrue(groupAssignment.getMembers().isEmpty());
+}
+
+@Test
+public void testFirstAssignmentTwoConsumersTwoTopicsSameSubscriptions() {
+// A -> T1, T3 // B -> T1, T3 // T1 -> 3 Partitions // T3 -> 2 
Partitions
+// Topics
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(topic1Name, 3));
+topics.put(topic3Uuid, new AssignmentTopicMetadata(topic3Name, 2));
+// Members
+Map members = new HashMap<>();
+// Consumer A
+List subscribedTopicsA = new 
ArrayList<>(Arrays.asList(topic1Uuid, topic3Uuid));
+members.put(consumerA, new AssignmentMemberSpec(Optional.empty(), 
Optional.empty(), subscribedTopicsA, new HashMap<>()));
+// Consumer B
+List subscribedTopicsB = new 
ArrayList<>(Arrays.asList(topic1Uuid, topic3Uuid));
+members.put(consumerB, new AssignmentMemberSpec(Optional.empty(), 
Optional.empty(), subscribedTopicsB, new HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment computedAssignment = assignor.assign(assignmentSpec);
+
+Map>> expectedAssignment = new HashMap<>();
+// Topic 1 Partitions Assignment
+expectedAssignment.computeIfAbsent(topic1Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Arrays.asList(0, 1)));
+expectedAssignment.computeIfAbsent(topic1Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(2)));
+// Topic 3 Partitions Assignment
+expectedAssignment.computeIfAbsent(topic3Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(0)));
+expectedAssignment.computeIfAbsent(topic3Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(1)));
+
+assertAssignment(expectedAssignment, computedAssignment);
+}
+
+@Test
+public void 
testFirstAssignmentThreeConsumersThreeTopicsDifferentSubscriptions() {
+// A -> T1, T2 // B -> T3 // C -> T2, T3 // T1 -> 3 Partitions // T2 
-> 3 Partitions // T3 -> 2 Partitions
+// Topics
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(topic1Name, 3));
+topics.put(topic2Uuid, new AssignmentTopicMetadata(topic1Name, 3));
+topics.put(topic3Uuid, new AssignmentTopicMetadata(topic3Name, 2));
+// Members
+Map members = new HashMap<>();
+// 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-03 Thread via GitHub


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


##
group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignorTest.java:
##
@@ -0,0 +1,389 @@
+/*
+ * 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.junit.jupiter.api.Test;
+
+import java.util.Map;
+import java.util.Set;
+
+
+
+import java.util.*;
+
+import static org.junit.jupiter.api.Assertions.*;
+
+public class ServerSideStickyRangeAssignorTest {
+
+private final ServerSideStickyRangeAssignor assignor = new 
ServerSideStickyRangeAssignor();
+
+private final String topic1Name = "topic1";
+private final Uuid topic1Uuid = Uuid.randomUuid();
+
+private final String topic2Name = "topic2";
+private final Uuid topic2Uuid = Uuid.randomUuid();
+
+private final String topic3Name = "topic3";
+private final Uuid topic3Uuid = Uuid.randomUuid();
+
+private final String consumerA = "A";
+private final String consumerB = "B";
+private final String consumerC = "C";
+
+@Test
+public void testOneConsumerNoTopic() {
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(topic1Name, 3));
+Map members = new HashMap<>();
+List subscribedTopics = new ArrayList<>();
+members.computeIfAbsent(consumerA, k -> new 
AssignmentMemberSpec(Optional.empty(), Optional.empty(), subscribedTopics, new 
HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment groupAssignment = assignor.assign(assignmentSpec);
+
+assertTrue(groupAssignment.getMembers().isEmpty());
+}
+
+@Test
+public void testFirstAssignmentTwoConsumersTwoTopicsSameSubscriptions() {
+// A -> T1, T3 // B -> T1, T3 // T1 -> 3 Partitions // T3 -> 2 
Partitions
+// Topics
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(topic1Name, 3));
+topics.put(topic3Uuid, new AssignmentTopicMetadata(topic3Name, 2));
+// Members
+Map members = new HashMap<>();
+// Consumer A
+List subscribedTopicsA = new 
ArrayList<>(Arrays.asList(topic1Uuid, topic3Uuid));
+members.put(consumerA, new AssignmentMemberSpec(Optional.empty(), 
Optional.empty(), subscribedTopicsA, new HashMap<>()));
+// Consumer B
+List subscribedTopicsB = new 
ArrayList<>(Arrays.asList(topic1Uuid, topic3Uuid));
+members.put(consumerB, new AssignmentMemberSpec(Optional.empty(), 
Optional.empty(), subscribedTopicsB, new HashMap<>()));
+
+AssignmentSpec assignmentSpec = new AssignmentSpec(members, topics);
+GroupAssignment computedAssignment = assignor.assign(assignmentSpec);
+
+Map>> expectedAssignment = new HashMap<>();
+// Topic 1 Partitions Assignment
+expectedAssignment.computeIfAbsent(topic1Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Arrays.asList(0, 1)));
+expectedAssignment.computeIfAbsent(topic1Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(2)));
+// Topic 3 Partitions Assignment
+expectedAssignment.computeIfAbsent(topic3Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(0)));
+expectedAssignment.computeIfAbsent(topic3Uuid, k -> new 
HashSet<>()).add(new HashSet<>(Collections.singletonList(1)));
+
+assertAssignment(expectedAssignment, computedAssignment);
+}
+
+@Test
+public void 
testFirstAssignmentThreeConsumersThreeTopicsDifferentSubscriptions() {
+// A -> T1, T2 // B -> T3 // C -> T2, T3 // T1 -> 3 Partitions // T2 
-> 3 Partitions // T3 -> 2 Partitions
+// Topics
+Map topics = new HashMap<>();
+topics.put(topic1Uuid, new AssignmentTopicMetadata(topic1Name, 3));
+topics.put(topic2Uuid, new AssignmentTopicMetadata(topic1Name, 3));

Review Comment:
   No we want all the mappings to be with Uuid



-- 
This is an automated message from the Apache Git Service.
To respond to the 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-03 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {

Review Comment:
   this is a method though so private is implicitly final right?



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-30 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {

Review Comment:
   sry my bad! I was thtinking about the putList and putSet
   



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {

Review Comment:
   Yep its a very specific type of pair so I custom made this, we could add it 
if its useful in other places as well



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {

Review Comment:
   since its already private, its treated as a final param so prolly not 
necessary here



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**

Review Comment:
   yess thanks !



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-27 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-27 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-27 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-27 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-27 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-27 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-27 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-27 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {
+List list = map.computeIfAbsent(key, k -> new ArrayList<>());
+list.add(value);
+}
+
+protected static  void putSet(Map> map, K key, V value) {
+Set set = map.computeIfAbsent(key, k -> new HashSet<>());
+set.add(value);
+}
+
+static class Pair {
+private final T first;
+private final U second;
+
+public Pair(T first, U second) {
+this.first = first;
+this.second = second;
+}
+
+public T getFirst() {
+return first;
+}
+
+public U getSecond() {
+return second;
+}
+
+@Override
+public String toString() {
+return "(" + first + ", " + second + ")";
+}
+}
+
+// Returns a map of the list of consumers per Topic (keyed by topicId)
+private Map> consumersPerTopic(AssignmentSpec 
assignmentSpec) {
+Map> mapTopicsToConsumers = new HashMap<>();
+Map membersData = assignmentSpec.members;
+
+for (Map.Entry memberEntry : 
membersData.entrySet()) {
+String memberId = memberEntry.getKey();
+AssignmentMemberSpec memberMetadata = memberEntry.getValue();
+List topics = new 
ArrayList<>(memberMetadata.subscribedTopics);
+for (Uuid topicId: topics) {
+putList(mapTopicsToConsumers, topicId, memberId);
+}
+}
+return mapTopicsToConsumers;
+}
+
+private Map> 
getAvailablePartitionsPerTopic(AssignmentSpec assignmentSpec, Map> assignedStickyPartitionsPerTopic) {
+Map> availablePartitionsPerTopic = new HashMap<>();
+Map topicsMetadata = 

[GitHub] [kafka] rreddy-22 commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-27 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/ServerSideStickyRangeAssignor.java:
##
@@ -0,0 +1,268 @@
+/*
+ * 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.
+ */
+
+/**
+ * The Server Side Sticky Range Assignor inherits properties of both the 
range assignor and the sticky assignor.
+ * Properties are :-
+ * 
+ *  1) Each consumer must get at least one partition per topic that it is 
subscribed to whenever the number of consumers is
+ *less than or equal to the number of partitions for that topic. (Range) 

+ *  2) Partitions should be assigned to consumers in a way that 
facilitates join operations where required. (Range) 
+ *This can only be done if the topics are co-partitioned in the first place
+ *Co-partitioned:-
+ *Two streams are co-partitioned if the following conditions are met:-
+ * ->The keys must have the same schemas
+ * ->The topics involved must have the same number of partitions
+ *  3) Consumers should retain as much as their previous assignment as 
possible. (Sticky) 
+ * 
+ * 
+ *
+ * The algorithm works mainly in 5 steps described below
+ * 
+ *  1) Get a map of the consumersPerTopic created using the member 
subscriptions.
+ *  2) Get a list of consumers (potentiallyUnfilled) that have not met the 
minimum required quota for assignment AND
+ * get a list of sticky partitions that we want to retain in the new 
assignment.
+ *  3) Add consumers from potentiallyUnfilled to Unfilled if they haven't 
met the total required quota = minQuota + (if necessary) extraPartition 
+ *  4) Get a list of available partitions by calculating the difference 
between total partitions and assigned sticky partitions 
+ *  5) Iterate through unfilled consumers and assign partitions from 
available partitions 
+ * 
+ * 
+ *
+ */
+package org.apache.kafka.coordinator.group.assignor;
+
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.HashMap;
+import java.util.Collections;
+
+import static java.lang.Math.min;
+
+public class ServerSideStickyRangeAssignor implements PartitionAssignor {
+
+public static final String RANGE_ASSIGNOR_NAME = "range-sticky";
+
+@Override
+public String name() {
+return RANGE_ASSIGNOR_NAME;
+}
+
+protected static  void putList(Map> map, K key, V value) {

Review Comment:
   It's specific to the data structures we're using for the assignor so I made 
it protected so only classes that use the same Maps can use it, to avoid 
confusion.



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-27 Thread via GitHub


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


##
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/assignor/AssignmentMemberSpec.java:
##
@@ -37,29 +39,28 @@ public class AssignmentMemberSpec {
 final Optional rackId;
 
 /**
- * The topics that the member is subscribed to.
+ * The topicIds of topics that the member is subscribed to.
  */
-final Collection subscribedTopics;
+final List subscribedTopics;

Review Comment:
   No its not necessary but it's easier to work with, is there a reason why we 
should use collection instead of list?



-- 
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 #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-27 Thread via GitHub


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


##
checkstyle/suppressions.xml:
##
@@ -320,6 +320,14 @@
 
 
+
+
+