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

2023-04-12 Thread via GitHub


jeffkbkim commented on code in PR #13443:
URL: https://github.com/apache/kafka/pull/13443#discussion_r1160974966


##
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] jeffkbkim commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-12 Thread via GitHub


jeffkbkim commented on code in PR #13443:
URL: https://github.com/apache/kafka/pull/13443#discussion_r1160941760


##
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:
   i think
   ```
   membersData.forEach( (memberId, assignmentMemberSpec) -> {
   
   })
   ```
   looks simpler. wdyt?



-- 
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] jeffkbkim commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-04-03 Thread via GitHub


jeffkbkim commented on code in PR #13443:
URL: https://github.com/apache/kafka/pull/13443#discussion_r1156251873


##
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, i'm referring to `AssignmentTopicMetadata(topic1Name, 3)`



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

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

2023-04-03 Thread via GitHub


jeffkbkim commented on code in PR #13443:
URL: https://github.com/apache/kafka/pull/13443#discussion_r1156090957


##
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:
   should this be topic2Name?



##

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

2023-03-29 Thread via GitHub


jeffkbkim commented on code in PR #13443:
URL: https://github.com/apache/kafka/pull/13443#discussion_r1152217710


##
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] jeffkbkim commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


jeffkbkim commented on code in PR #13443:
URL: https://github.com/apache/kafka/pull/13443#discussion_r1152204557


##
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] jeffkbkim commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


jeffkbkim commented on code in PR #13443:
URL: https://github.com/apache/kafka/pull/13443#discussion_r1152199939


##
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] jeffkbkim commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


jeffkbkim commented on code in PR #13443:
URL: https://github.com/apache/kafka/pull/13443#discussion_r1152198768


##
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:
   can we make these private?



-- 
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] jeffkbkim commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-29 Thread via GitHub


jeffkbkim commented on code in PR #13443:
URL: https://github.com/apache/kafka/pull/13443#discussion_r1152197788


##
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:
   Collection allows iterating over elements which is the only use i see with 
subscribedTopics. it makes it more generic
   
   there's no need to use List 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] jeffkbkim commented on a diff in pull request #13443: KAFKA-14514: Add Server Side Sticky Range Assignor (KIP-848)

2023-03-24 Thread via GitHub


jeffkbkim commented on code in PR #13443:
URL: https://github.com/apache/kafka/pull/13443#discussion_r1146805063


##
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:
   is it necessary to use List?



##
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:
   do these methods need to be protected?



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