From 4caa11212ef7e8d27f3c1ebf5f16b21e89c56772 Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Thu, 18 Feb 2016 05:44:17 -0800 Subject: [PATCH 1/9] KAFKA-2273: Sticky partition assignment strategy (KIP-54) This PR implements a new partition assignment strategy called "sticky", as described in KIP-54, and it's purpose is to balance partitions across consumers in a way that minimizes moving partitions around, or in other words, preserving partition assignments as much as possible. --- .../clients/consumer/StickyAssignor.java | 480 +++++++++++++++ .../internals/AbstractPartitionAssignor.java | 9 +- .../clients/consumer/StickyAssignorTest.java | 571 ++++++++++++++++++ .../kafka/api/PlaintextConsumerTest.scala | 69 ++- 4 files changed, 1126 insertions(+), 3 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java new file mode 100644 index 000000000000..b43904a59c1c --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java @@ -0,0 +1,480 @@ +/* + * 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.clients.consumer; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor; +import org.apache.kafka.common.TopicPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The sticky assignor serves two purposes. First, it guarantees an assignment that is as balanced as possible, meaning either: + * - the numbers of topic partitions assigned to consumers differ by at most one; or + * - each consumer that has 2+ fewer topic partitions than some other consumer cannot get any of those topic partitions transfered to it. + * + * Starting fresh it would work by distributing the partitions over consumers as evenly as possible. Even though this may sound similar to + * how round robin assignor works, the second example below shows that it is not. + * During a reassignment it would perform the reassignment in such a way that in the new assignment + * 1. topic partitions are still distributed as evenly as possible, and + * 2. topic partitions stay with their previously assigned consumers as much as possible. + * Of course, the first goal above takes precedence over the second one. + * + * Example 1. Suppose there are three consumers C0, C1, C2, four topics t0, t1, t2, t3, and each topic has 2 partitions, + * resulting in partitions t0p0, t0p1, t1p0, t1p1, t2p0, t2p1, t3p0, t3p1. Each consumer is subscribed to all three topics. + * + * The assignment with both sticky and round robin assignors will be: + * C0: [t0p0, t1p1, t3p0] + * C1: [t0p1, t2p0, t3p1] + * C2: [t1p0, t2p1] + * + * Now, let's assume C1 is removed and a reassignment is about to happen. The round robin assignor would produce: + * C0: [t0p0, t1p0, t2p0, t3p0] + * C2: [t0p1, t1p1, t2p1, t3p1] + * + * while the sticky assignor would result in: + * C0 [t0p0, t1p1, t3p0, t2p0] + * C2 [t1p0, t2p1, t0p1, t3p1] + * preserving all the previous assignments (unlike the round robin assignor). + * + * Example 2. There are three consumers C0, C1, C2, and three topics t0, t1, t2, with 1, 2, and 3 partitions respectively. + * Therefore, the partitions are t0p0, t1p0, t1p1, t2p0, t2p1, t2p2. + * C0 is subscribed to t0; C1 is subscribed to t0, t1; and C2 is subscribed to t0, t1, t2. + * + * The round robin assignor would come up with the following assignment: + * C0 [t0p0] + * C1 [t1p0] + * C2 [t1p1, t2p0, t2p1, t2p2] + * + * which is not as balanced as the assignment suggested by sticky assignor: + * C0 [t0p0] + * C1 [t1p0, t1p1] + * C2 [t2p0, t2p1, t2p2] + * + * Now, if consumer C0 is removed, these two assignors would produce the following assignments. + * Round Robin (preserves 3 partition assignments): + * C1 [t0p0, t1p1] + * C2 [t1p0, t2p0, t2p1, t2p2] + * + * Sticky (preserves 5 partition assignments): + * C1 [t1p0, t1p1, t0p0] + * C2 [t2p0, t2p1, t2p2] + * + */ +public class StickyAssignor extends AbstractPartitionAssignor { + + private static final Logger log = LoggerFactory.getLogger(StickyAssignor.class); + protected Map> currentAssignment = new HashMap<>(); + + private void deepCopy(Map> source, Map> dest) { + dest.clear(); + for (Entry> entry: source.entrySet()) + dest.put(entry.getKey(), new ArrayList(entry.getValue())); + } + + private Map> deepCopy(Map> assignment) { + Map> copy = new HashMap<>(); + deepCopy(assignment, copy); + return copy; + } + + private HashMap deepCopy(HashMap source) { + HashMap copy = new HashMap<>(); + copy.putAll(source); + return copy; + } + + public Map> assign(Map partitionsPerTopic, + Map> subscriptions) { + // make a deep copy of currentAssignment + Map> oldAssignment = deepCopy(currentAssignment); + + // a mapping of all topic partitions to all consumers that can be assigned to them + final HashMap> partition2AllPotentialConsumers = new HashMap<>(); + // a mapping of all consumers to all potential topic partitions that can be assigned to them + final HashMap> consumer2AllPotentialPartitions = new HashMap<>(); + + // initialize partition2AllPotentialConsumers and consumer2AllPotentialPartitions in the following two for loops + Set topics = partitionsPerTopic.keySet(); + for (String topic: topics) { + int partitions = partitionsPerTopic.get(topic); + for (int i = 0; i < partitions; ++i) + partition2AllPotentialConsumers.put(new TopicPartition(topic, i), new ArrayList()); + } + + Set consumers = subscriptions.keySet(); + for (String consumer: consumers) { + consumer2AllPotentialPartitions.put(consumer, new ArrayList()); + List consumerTopics = subscriptions.get(consumer); + for (String topic: consumerTopics) { + int partitions = partitionsPerTopic.get(topic); + for (int i = 0; i < partitions; ++i) { + TopicPartition topicPartition = new TopicPartition(topic, i); + consumer2AllPotentialPartitions.get(consumer).add(topicPartition); + partition2AllPotentialConsumers.get(topicPartition).add(consumer); + } + } + // add this consumer to currentAssignment (with an empty topic partition assignment) if it does not already exist + if (!currentAssignment.containsKey(consumer)) + currentAssignment.put(consumer, new ArrayList()); + } + + // a mapping of partition to current consumer + HashMap currentPartitionConsumer = new HashMap<>(); + for (String consumer: currentAssignment.keySet()) + for (TopicPartition topicPartition: currentAssignment.get(consumer)) + currentPartitionConsumer.put(topicPartition, consumer); + + // an ascending sorted set of topic partitions based on how many consumers can potentially use them + TreeSet sortedAllPartitions = new TreeSet<>(new PartitionComparator(partition2AllPotentialConsumers)); + sortedAllPartitions.addAll(partition2AllPotentialConsumers.keySet()); + + // all partitions that need to be assigned (initially set to all partitions but adjusted in the following loop) + TreeSet unassignedPartitions = new TreeSet<>(sortedAllPartitions); + for (Iterator>> it = currentAssignment.entrySet().iterator(); it.hasNext();) { + Map.Entry> entry = it.next(); + if (!subscriptions.containsKey(entry.getKey())) { + // if a consumer that existed before (and had some partition assignments) is now removed, remove it from currentAssignment + for (TopicPartition topicPartition: entry.getValue()) + currentPartitionConsumer.remove(topicPartition); + it.remove(); + } else { + // otherwise (the consumer still exists) + for (Iterator partitionIter = entry.getValue().iterator(); partitionIter.hasNext();) { + TopicPartition partition = partitionIter.next(); + if (!partition2AllPotentialConsumers.containsKey(partition)) { + // if this topic partition of this consumer no longer exists remove it from currentAssignment of the consumer + partitionIter.remove(); + currentPartitionConsumer.remove(partition); + } else + // otherwise, remove the topic partition from those that need to be assigned (because it is already assigned + // and we would want to preserve that assignment as much as possible) + unassignedPartitions.remove(partition); + } + } + } + // at this point we have preserved all valid topic partition to consumer assignments and removed + // all invalid topic partitions and invalid consumers. Now we need to assign unassignedPartitions + // to consumers so that the topic partition assignments are as balanced as possible. + + // an ascending sorted set of consumers based on how many topic partitions are already assigned to them + TreeSet sortedCurrentSubscriptions = new TreeSet<>(new SubscriptionComparator(currentAssignment)); + sortedCurrentSubscriptions.addAll(currentAssignment.keySet()); + + balance(sortedAllPartitions, unassignedPartitions, sortedCurrentSubscriptions, consumer2AllPotentialPartitions, + partition2AllPotentialConsumers, oldAssignment, currentPartitionConsumer); + return currentAssignment; + } + + + @Override + protected ByteBuffer getUserData() { + try { + ByteArrayOutputStream bos = new ByteArrayOutputStream(2048); + ObjectOutputStream oos = new ObjectOutputStream(bos); + oos.writeObject(currentAssignment); + return ByteBuffer.wrap(bos.toByteArray()); + } catch (Exception ioex) { + log.error("Failed to serialize currentAssignment", ioex); + return super.getUserData(); + } + } + + @SuppressWarnings("unchecked") + @Override + public void onAssignment(Assignment assignment) { + try { + byte[] array = new byte[assignment.userData().remaining()]; + assignment.userData().get(array); + ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(array)); + currentAssignment = (Map>) ois.readObject(); + log.debug("Got currentAssignment = " + currentAssignment); + } catch (IOException | ClassNotFoundException exc) { + log.error("Failed to deserialize assignment userdata into currentAssignment", exc); + currentAssignment.clear(); + } + } + + @Override + public String name() { + return "sticky"; + } + + /** + * determine if the current assignment is a balanced one + * + * @param sortedCurrentSubscriptions: an ascending sorted set of consumers based on how many topic partitions are already assigned to them + * @param allSubscriptions: a mapping of all consumers to all potential topic partitions that can be assigned to them + * @return + */ + protected boolean isBalanced(TreeSet sortedCurrentSubscriptions, Map> allSubscriptions) { + int min = currentAssignment.get(sortedCurrentSubscriptions.first()).size(); + int max = currentAssignment.get(sortedCurrentSubscriptions.last()).size(); + if (min >= max - 1) + // if minimum and maximum numbers of partitions assigned to consumers differ by at most one return true + return true; + + // create a mapping from partitions to the consumer assigned to them + final HashMap allPartitions = new HashMap<>(); + Set>> assignments = currentAssignment.entrySet(); + for (Map.Entry> entry: assignments) { + List topicPartitions = entry.getValue(); + for (TopicPartition topicPartition: topicPartitions) { + if (allPartitions.containsKey(topicPartition)) + log.error(topicPartition + " is assigned to more than one consumer."); + allPartitions.put(topicPartition, entry.getKey()); + } + } + + // for each consumer that does not have all the topic partitions it can get make sure none of the topic partitions it + // could but did not get cannot be moved to it (because that would break the balance) + for (String consumer: sortedCurrentSubscriptions) { + List consumerPartitions = currentAssignment.get(consumer); + int consumerPartitionCount = consumerPartitions.size(); + + // skip if this consumer already has all the topic partitions it can get + if (consumerPartitionCount == allSubscriptions.get(consumer).size()) + continue; + + // otherwise make sure it cannot get any more + List potentialTopicPartitions = allSubscriptions.get(consumer); + for (TopicPartition topicPartition: potentialTopicPartitions) { + if (!currentAssignment.get(consumer).contains(topicPartition)) { + String otherConsumer = allPartitions.get(topicPartition); + int otherConsumerPartitionCount = currentAssignment.get(otherConsumer).size(); + if (consumerPartitionCount < otherConsumerPartitionCount) { + log.debug(topicPartition + " can be moved from consumer " + otherConsumer + " to consumer " + consumer + " for a more balanced assignment."); + return false; + } + } + } + } + return true; + } + + /** + * @return the balance score of the given assignment, as the sum of assigned partitions size difference of all consumer pairs. + * A perfectly balanced assignment (with all consumers getting the same number of partitions) has a balance score of 0. + * Lower balance score indicates a more balanced assignment. + */ + protected int getBalanceScore(Map> assignment) { + int score = 0; + Set consumers = assignment.keySet(); + Set consumersCopy = new HashSet<>(consumers); + for (String consumer: consumers) { + consumersCopy.remove(consumer); + int consumerAssignmentSize = assignment.get(consumer).size(); + for (String otherKey: consumersCopy) + score += Math.abs(consumerAssignmentSize - assignment.get(otherKey).size()); + } + return score; + } + + /** + * @return the consumer to which the given partition is assigned. The assignment should improve the overall balance + * of the partition assignments to consumers. + */ + private String assignPartition(TopicPartition partition, TreeSet sortedCurrentSubscriptions, + HashMap> consumer2AllPotentialPartitions, HashMap currentPartitionConsumer) { + for (String consumer: sortedCurrentSubscriptions) { + if (consumer2AllPotentialPartitions.get(consumer).contains(partition)) { + sortedCurrentSubscriptions.remove(consumer); + currentAssignment.get(consumer).add(partition); + currentPartitionConsumer.put(partition, consumer); + sortedCurrentSubscriptions.add(consumer); + return consumer; + } + } + return null; + } + + /** + * Remove partition from partition assignments of the given consumer + */ + private void deassignPartition(TopicPartition partition, String consumer, TreeSet sortedCurrentSubscriptions, HashMap currentPartitionConsumer) { + sortedCurrentSubscriptions.remove(consumer); + currentAssignment.get(consumer).remove(partition); + currentPartitionConsumer.remove(partition); + sortedCurrentSubscriptions.add(consumer); + } + + private boolean canParticipateInReassignment(TopicPartition partition, HashMap> partition2AllPotentialConsumers) { + // if a partition has two or more potential consumers it is subject to reassignment. + return partition2AllPotentialConsumers.get(partition).size() >= 2; + } + + private boolean canParticipateInReassignment(String consumer, + HashMap> consumer2AllPotentialPartitions, + HashMap> partition2AllPotentialConsumers) { + List currentPartitions = currentAssignment.get(consumer); + int currentAssignmentSize = currentPartitions.size(); + int maxAssignmentSize = consumer2AllPotentialPartitions.get(consumer).size(); + if (currentAssignmentSize > maxAssignmentSize) + log.error("The consumer " + consumer + " is assigned more partitions than the maximum possible."); + + if (currentAssignmentSize < maxAssignmentSize) + // if a consumer is not assigned all its potential partitions it is subject to reassignment + return true; + + for (TopicPartition partition: currentPartitions) + // if any of the partitions assigned to a consumer is subject to reassignment the consumer itself + // is subject to reassignment + if (canParticipateInReassignment(partition, partition2AllPotentialConsumers)) + return true; + + return false; + } + + /** + * Balance the current assignment using the data structures created in the assign(...) method above. + */ + private void balance(TreeSet sortedAllPartitions, TreeSet unassignedPartitions, TreeSet sortedCurrentSubscriptions, + HashMap> consumer2AllPotentialPartitions, HashMap> partition2AllPotentialConsumers, + Map> oldAssignment, HashMap currentPartitionConsumer) { + boolean initializing = currentAssignment.get(sortedCurrentSubscriptions.last()).isEmpty(); + boolean reassignmentPerformed = false; + + // assign all unassigned partitions + while (!unassignedPartitions.isEmpty()) { + TopicPartition partition = unassignedPartitions.pollFirst(); + + // skip if there is no potential consumer for the partition + if (partition2AllPotentialConsumers.get(partition).isEmpty()) + continue; + + assignPartition(partition, sortedCurrentSubscriptions, consumer2AllPotentialPartitions, currentPartitionConsumer); + } + + // narrow down the reassignment scope to only those partitions that can actually be reassigned + Set reassignablePartitions = new HashSet<>(partition2AllPotentialConsumers.keySet()); + Set fixedPartitions = new HashSet<>(); + for (TopicPartition partition: reassignablePartitions) + if (!canParticipateInReassignment(partition, partition2AllPotentialConsumers)) + fixedPartitions.add(partition); + reassignablePartitions.removeAll(fixedPartitions); + sortedAllPartitions.removeAll(fixedPartitions); + + // narrow down the reassignment scope to only those consumers that are subject to reassignment + Map> fixedAssignments = new HashMap<>(); + for (String consumer: consumer2AllPotentialPartitions.keySet()) + if (!canParticipateInReassignment(consumer, consumer2AllPotentialPartitions, partition2AllPotentialConsumers)) { + sortedCurrentSubscriptions.remove(consumer); + fixedAssignments.put(consumer, currentAssignment.remove(consumer)); + } + + // create a deep copy of the current assignment so we can revert to it if we do not get a more balanced assignment later + Map> preBalanceAssignment = deepCopy(currentAssignment); + HashMap preBalancePartitionConsumers = deepCopy(currentPartitionConsumer); + + // repeat reassignment until no partition can be moved to improve the balance + boolean modified = true; + while (modified) { + modified = false; + // reassign all reassignable partitions (starting from the partition with least potential consumers and if needed) + // until the full list is processed or a balance is achieved + Iterator partitionIterator = sortedAllPartitions.iterator(); + while (partitionIterator.hasNext() && !isBalanced(sortedCurrentSubscriptions, consumer2AllPotentialPartitions)) { + TopicPartition partition = partitionIterator.next(); + + // the partition must have at least two consumers + if (partition2AllPotentialConsumers.get(partition).size() <= 1) + log.error("Expected more than one potential consumer for partition '" + partition + "'"); + + // the partition must have a current consumer + String consumer = currentPartitionConsumer.get(partition); + if (consumer == null) + log.error("Expected partition '" + partition + "' to be assigned to a consumer"); + + // check if a better-suited consumer exist for the partition; if so, reassign it + for (String otherConsumer: partition2AllPotentialConsumers.get(partition)) + if (currentAssignment.get(consumer).size() > currentAssignment.get(otherConsumer).size() + 1) { + // de-assign partition from its current consumer + deassignPartition(partition, consumer, sortedCurrentSubscriptions, currentPartitionConsumer); + // reassign the partition to an eligible consumer with fewest assignments + assignPartition(partition, sortedCurrentSubscriptions, consumer2AllPotentialPartitions, currentPartitionConsumer); + reassignmentPerformed = true; + modified = true; + break; + } + } + } + + // if we are not preserving existing assignments and we have made changes to the current assignment + // make sure we are getting a more balanced assignment; otherwise, revert to previous assignment + if (!initializing && reassignmentPerformed && getBalanceScore(currentAssignment) >= getBalanceScore(preBalanceAssignment)) { + deepCopy(preBalanceAssignment, currentAssignment); + currentPartitionConsumer = preBalancePartitionConsumers; + } + + // add the fixed assignments (those that could not change) back + for (String consumer: fixedAssignments.keySet()) { + currentAssignment.put(consumer, fixedAssignments.get(consumer)); + sortedCurrentSubscriptions.add(consumer); + } + fixedAssignments.clear(); + } + + private static class PartitionComparator implements Comparator { + private Map> map; + + PartitionComparator(Map> map) { + this.map = map; + } + + @Override + public int compare(TopicPartition o1, TopicPartition o2) { + int ret = map.get(o1).size() - map.get(o2).size(); + if (ret == 0) { + ret = o1.topic().compareTo(o2.topic()); + if (ret == 0) + ret = o1.partition() - o2.partition(); + } + return ret; + } + } + protected static class SubscriptionComparator implements Comparator { + private Map> map; + + SubscriptionComparator(Map> map) { + this.map = map; + } + + @Override + public int compare(String o1, String o2) { + int ret = map.get(o1).size() - map.get(o2).size(); + if (ret == 0) + ret = o1.compareTo(o2); + return ret; + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java index 5c97693e91fc..7ae7fa8455db 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java @@ -21,6 +21,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -71,13 +72,17 @@ public Map assign(Cluster metadata, Map> rawAssignments = assign(partitionsPerTopic, topicSubscriptions); - // this class has maintains no user data, so just wrap the results + // wrap the results along with any available user data Map assignments = new HashMap<>(); for (Map.Entry> assignmentEntry : rawAssignments.entrySet()) - assignments.put(assignmentEntry.getKey(), new Assignment(assignmentEntry.getValue())); + assignments.put(assignmentEntry.getKey(), new Assignment(assignmentEntry.getValue(), getUserData())); return assignments; } + protected ByteBuffer getUserData() { + return ByteBuffer.wrap(new byte[0]); + } + @Override public void onAssignment(Assignment assignment) { // this assignor maintains no internal state, so nothing to do diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java new file mode 100644 index 000000000000..8196089b7629 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java @@ -0,0 +1,571 @@ +/** + * 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.clients.consumer; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +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.Map.Entry; +import java.util.Random; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.kafka.common.TopicPartition; +import org.junit.Test; + +public class StickyAssignorTest { + + private StickyAssignor assignor = new StickyAssignor(); + + @Test + public void testOneConsumerNoTopic() { + String consumerId = "consumer"; + + Map partitionsPerTopic = new HashMap<>(); + Map> subscriptions = Collections.singletonMap(consumerId, Collections.emptyList()); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(Collections.singleton(consumerId), assignment.keySet()); + assertTrue(assignment.get(consumerId).isEmpty()); + + assertTrue(isFullyBalanced(assignment)); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testOneConsumerNonexistentTopic() { + String topic = "topic"; + String consumerId = "consumer"; + + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic, 0); + Map> subscriptions = Collections.singletonMap(consumerId, topics(topic)); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + + assertEquals(Collections.singleton(consumerId), assignment.keySet()); + assertTrue(assignment.get(consumerId).isEmpty()); + + assertTrue(isFullyBalanced(assignment)); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testOneConsumerOneTopic() { + String topic = "topic"; + String consumerId = "consumer"; + + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic, 3); + Map> subscriptions = Collections.singletonMap(consumerId, topics(topic)); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(Arrays.asList( + new TopicPartition(topic, 0), + new TopicPartition(topic, 1), + new TopicPartition(topic, 2)), assignment.get(consumerId)); + + assertTrue(isFullyBalanced(assignment)); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testOnlyAssignsPartitionsFromSubscribedTopics() { + String topic = "topic"; + String otherTopic = "other"; + String consumerId = "consumer"; + + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic, 3); + partitionsPerTopic.put(otherTopic, 3); + Map> subscriptions = Collections.singletonMap(consumerId, topics(topic)); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(Arrays.asList( + new TopicPartition(topic, 0), + new TopicPartition(topic, 1), + new TopicPartition(topic, 2)), assignment.get(consumerId)); + + assertTrue(isFullyBalanced(assignment)); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testOneConsumerMultipleTopics() { + String topic1 = "topic1"; + String topic2 = "topic2"; + String consumerId = "consumer"; + + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic1, 1); + partitionsPerTopic.put(topic2, 2); + Map> subscriptions = Collections.singletonMap(consumerId, topics(topic1, topic2)); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(Arrays.asList( + new TopicPartition(topic1, 0), + new TopicPartition(topic2, 0), + new TopicPartition(topic2, 1)), assignment.get(consumerId)); + + assertTrue(isFullyBalanced(assignment)); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testTwoConsumersOneTopicOnePartition() { + String topic = "topic"; + String consumer1 = "consumer1"; + String consumer2 = "consumer2"; + + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic, 1); + + Map> subscriptions = new HashMap<>(); + subscriptions.put(consumer1, topics(topic)); + subscriptions.put(consumer2, topics(topic)); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(Arrays.asList(new TopicPartition(topic, 0)), assignment.get(consumer1)); + assertEquals(Collections.emptyList(), assignment.get(consumer2)); + + assertTrue(isFullyBalanced(assignment)); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testTwoConsumersOneTopicTwoPartitions() { + String topic = "topic"; + String consumer1 = "consumer1"; + String consumer2 = "consumer2"; + + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic, 2); + + Map> subscriptions = new HashMap<>(); + subscriptions.put(consumer1, topics(topic)); + subscriptions.put(consumer2, topics(topic)); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(Arrays.asList(new TopicPartition(topic, 0)), assignment.get(consumer1)); + assertEquals(Arrays.asList(new TopicPartition(topic, 1)), assignment.get(consumer2)); + + assertTrue(isFullyBalanced(assignment)); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testMultipleConsumersMixedTopicSubscriptions() { + String topic1 = "topic1"; + String topic2 = "topic2"; + String consumer1 = "consumer1"; + String consumer2 = "consumer2"; + String consumer3 = "consumer3"; + + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic1, 3); + partitionsPerTopic.put(topic2, 2); + + Map> subscriptions = new HashMap<>(); + subscriptions.put(consumer1, topics(topic1)); + subscriptions.put(consumer2, topics(topic1, topic2)); + subscriptions.put(consumer3, topics(topic1)); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(Arrays.asList( + new TopicPartition(topic1, 0), + new TopicPartition(topic1, 2)), assignment.get(consumer1)); + assertEquals(Arrays.asList( + new TopicPartition(topic2, 0), + new TopicPartition(topic2, 1)), assignment.get(consumer2)); + assertEquals(Arrays.asList( + new TopicPartition(topic1, 1)), assignment.get(consumer3)); + + assertTrue(isFullyBalanced(assignment)); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testTwoConsumersTwoTopicsSixPartitions() { + String topic1 = "topic1"; + String topic2 = "topic2"; + String consumer1 = "consumer1"; + String consumer2 = "consumer2"; + + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic1, 3); + partitionsPerTopic.put(topic2, 3); + + Map> subscriptions = new HashMap<>(); + subscriptions.put(consumer1, topics(topic1, topic2)); + subscriptions.put(consumer2, topics(topic1, topic2)); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(Arrays.asList( + new TopicPartition(topic1, 0), + new TopicPartition(topic1, 2), + new TopicPartition(topic2, 1)), assignment.get(consumer1)); + assertEquals(Arrays.asList( + new TopicPartition(topic1, 1), + new TopicPartition(topic2, 0), + new TopicPartition(topic2, 2)), assignment.get(consumer2)); + + assertTrue(isFullyBalanced(assignment)); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testAddRemoveConsumerOneTopic() { + String topic = "topic"; + String consumer1 = "consumer"; + + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic, 3); + Map> subscriptions = new HashMap<>(); + subscriptions.put(consumer1, topics(topic)); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(Arrays.asList( + new TopicPartition(topic, 0), + new TopicPartition(topic, 1), + new TopicPartition(topic, 2)), assignment.get(consumer1)); + assertTrue(isFullyBalanced(assignment)); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + + String consumer2 = "consumer2"; + subscriptions.put(consumer2, topics(topic)); + Map> assignment2 = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(Arrays.asList(new TopicPartition(topic, 1), new TopicPartition(topic, 2)), assignment2.get(consumer1)); + assertEquals(Arrays.asList(new TopicPartition(topic, 0)), assignment2.get(consumer2)); + assertTrue(isFullyBalanced(assignment2)); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + + subscriptions.remove(consumer1); + Map> assignment3 = assignor.assign(partitionsPerTopic, subscriptions); + assertTrue(assignment3.get(consumer2).contains(new TopicPartition(topic, 0))); + assertTrue(assignment3.get(consumer2).contains(new TopicPartition(topic, 1))); + assertTrue(assignment3.get(consumer2).contains(new TopicPartition(topic, 2))); + assertTrue(isFullyBalanced(assignment3)); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + /** + * This unit test performs sticky assignment for a scenario that round robin assignor handles poorly. + * Topics (partitions per topic): topic1 (2), topic2 (1), topic3 (2), topic4 (1), topic5 (2) + * Subscriptions: + * - consumer1: topic1, topic2, topic3, topic4, topic5 + * - consumer2: topic1, topic3, topic5 + * - consumer3: topic1, topic3, topic5 + * - consumer4: topic1, topic2, topic3, topic4, topic5 + * Round Robin Assignment Result: + * - consumer1: topic1-0, topic3-0, topic5-0 + * - consumer2: topic1-1, topic3-1, topic5-1 + * - consumer3: + * - consumer4: topic2-0, topic4-0 + * Sticky Assignment Result: + * - consumer1: topic2-0, topic3-0 + * - consumer2: topic1-0, topic3-1 + * - consumer3: topic1-1, topic5-0 + * - consumer4: topic4-0, topic5-1 + */ + @Test + public void testPoorRoundRobinAssignmentScenario() { + Map partitionsPerTopic = new HashMap<>(); + for (int i = 1; i <= 5; i++) + partitionsPerTopic.put(String.format("topic%d", i), (i % 2) + 1); + + Map> subscriptions = new HashMap<>(); + subscriptions.put("consumer1", Arrays.asList("topic1", "topic2", "topic3", "topic4", "topic5")); + subscriptions.put("consumer2", Arrays.asList("topic1", "topic3", "topic5")); + subscriptions.put("consumer3", Arrays.asList("topic1", "topic3", "topic5")); + subscriptions.put("consumer4", Arrays.asList("topic1", "topic2", "topic3", "topic4", "topic5")); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testAddRemoveTopicTwoConsumers() { + String topic = "topic"; + String consumer1 = "consumer"; + String consumer2 = "consumer2"; + + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put(topic, 3); + Map> subscriptions = new HashMap<>(); + subscriptions.put(consumer1, topics(topic)); + subscriptions.put(consumer2, topics(topic)); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(Arrays.asList(new TopicPartition(topic, 0), new TopicPartition(topic, 2)), assignment.get(consumer1)); + assertEquals(Arrays.asList(new TopicPartition(topic, 1)), assignment.get(consumer2)); + assertTrue(isFullyBalanced(assignment)); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + + String topic2 = "topic2"; + partitionsPerTopic.put(topic2, 3); + subscriptions.put(consumer1, topics(topic, topic2)); + subscriptions.put(consumer2, topics(topic, topic2)); + Map> assignment2 = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(Arrays.asList(new TopicPartition(topic, 0), new TopicPartition(topic, 2), new TopicPartition(topic2, 1)), assignment2.get(consumer1)); + assertEquals(Arrays.asList(new TopicPartition(topic, 1), new TopicPartition(topic2, 0), new TopicPartition(topic2, 2)), assignment2.get(consumer2)); + assertTrue(isFullyBalanced(assignment2)); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + + partitionsPerTopic.remove(topic); + subscriptions.put(consumer1, topics(topic2)); + subscriptions.put(consumer2, topics(topic2)); + Map> assignment3 = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(Arrays.asList(new TopicPartition(topic2, 1)), assignment3.get(consumer1)); + assertEquals(Arrays.asList(new TopicPartition(topic2, 0), new TopicPartition(topic2, 2)), assignment3.get(consumer2)); + assertTrue(isFullyBalanced(assignment3)); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testReassignmentAfterOneConsumerLeaves() { + Map partitionsPerTopic = new HashMap<>(); + for (int i = 1; i < 20; i++) + partitionsPerTopic.put(String.format("topic%02d", i), i); + + Map> subscriptions = new HashMap<>(); + for (int i = 1; i < 20; i++) { + List topics = new ArrayList(); + for (int j = 1; j <= i; j++) + topics.add(String.format("topic%02d", j)); + subscriptions.put(String.format("consumer%02d", i), topics); + } + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + + subscriptions.remove("consumer10"); + assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testSameSubscriptions() { + Map partitionsPerTopic = new HashMap<>(); + for (int i = 1; i < 15; i++) + partitionsPerTopic.put(String.format("topic%02d", i), i); + + Map> subscriptions = new HashMap<>(); + for (int i = 1; i < 9; i++) { + List topics = new ArrayList(); + for (int j = 1; j <= partitionsPerTopic.size(); j++) + topics.add(String.format("topic%02d", j)); + subscriptions.put(String.format("consumer%02d", i), topics); + } + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + + subscriptions.remove("consumer05"); + assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testLargeAssignmentWithMultipleConsumersLeaving() { + Random rand = new Random(); + int topicCount = 400; + int consumerCount = 1000; + + Map partitionsPerTopic = new HashMap<>(); + for (int i = 0; i < topicCount; i++) + partitionsPerTopic.put(String.format("t%d", i), rand.nextInt(10) + 1); + + Map> subscriptions = new HashMap<>(); + for (int i = 0; i < consumerCount; i++) { + List topics = new ArrayList(); + for (int j = 0; j < rand.nextInt(20); j++) + topics.add(String.format("t%d", rand.nextInt(topicCount))); + subscriptions.put(String.format("c%d", i), topics); + } + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + + for (int i = 0; i < 100; ++i) { + String c = String.format("c%d", rand.nextInt(consumerCount)); + subscriptions.remove(c); + } + + assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testNewSubscription() { + Map partitionsPerTopic = new HashMap<>(); + for (int i = 1; i < 5; i++) + partitionsPerTopic.put(String.format("topic%02d", i), 1); + + Map> subscriptions = new HashMap<>(); + for (int i = 0; i < 3; i++) { + List topics = new ArrayList(); + for (int j = i; j <= 3 * i - 2; j++) + topics.add(String.format("topic%02d", j)); + subscriptions.put(String.format("consumer%02d", i), topics); + } + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + + subscriptions.get("consumer00").add("topic01"); + + assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testMoveExistingAssignments() { + Map partitionsPerTopic = new HashMap<>(); + for (int i = 1; i <= 6; i++) + partitionsPerTopic.put(String.format("topic%02d", i), 1); + + Map> subscriptions = new HashMap<>(); + subscriptions.put("consumer01", topics("topic01", "topic02")); + subscriptions.put("consumer02", topics("topic01", "topic02", "topic03", "topic04")); + subscriptions.put("consumer03", topics("topic02", "topic03", "topic04", "topic05", "topic06")); + + assignor.currentAssignment.put("consumer01", new ArrayList<>(Arrays.asList(new TopicPartition("topic01", 0)))); + assignor.currentAssignment.put("consumer02", new ArrayList<>(Arrays.asList(new TopicPartition("topic02", 0), new TopicPartition("topic03", 0)))); + assignor.currentAssignment.put("consumer03", new ArrayList<>(Arrays.asList(new TopicPartition("topic04", 0), new TopicPartition("topic05", 0), new TopicPartition("topic06", 0)))); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + + @Test + public void testStickiness() { + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put("topic01", 3); + Map> subscriptions = new HashMap<>(); + subscriptions.put("consumer01", topics("topic01")); + subscriptions.put("consumer02", topics("topic01")); + subscriptions.put("consumer03", topics("topic01")); + subscriptions.put("consumer04", topics("topic01")); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + Map partitionsAssigned = new HashMap<>(); + + Set>> assignments = assignment.entrySet(); + for (Map.Entry> entry: assignments) { + String consumer = entry.getKey(); + List topicPartitions = entry.getValue(); + int size = topicPartitions.size(); + assertTrue("Consumer " + consumer + " is assigned more topic partitions than expected.", size <= 1); + if (size == 1) + partitionsAssigned.put(consumer, topicPartitions.get(0)); + } + + // removing the potential group leader + subscriptions.remove("consumer01"); + + assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + + assignments = assignment.entrySet(); + for (Map.Entry> entry: assignments) { + String consumer = entry.getKey(); + List topicPartitions = entry.getValue(); + assertEquals("Consumer " + consumer + " is assigned more topic partitions than expected.", 1, topicPartitions.size()); + assertTrue("Stickiness was not honored for consumer " + consumer, + (!partitionsAssigned.containsKey(consumer)) || (assignment.get(consumer).contains(partitionsAssigned.get(consumer)))); + } + } + + public static List topics(String... topics) { + return Arrays.asList(topics); + } + + public static TopicPartition tp(String topic, int partition) { + return new TopicPartition(topic, partition); + } + + private boolean isFullyBalanced(Map> assignment) { + int min = Integer.MAX_VALUE; + int max = Integer.MIN_VALUE; + for (List topicPartitions: assignment.values()) { + int size = topicPartitions.size(); + if (size < min) + min = size; + if (size > max) + max = size; + } + return max - min <= 1; + } + + /** + * Verify that either: + * - the given assignment is fully balanced (the numbers of topic partitions assigned to consumers differ by at most one), or + * - there is no topic partition that can be moved from one consumer to another that has 2+ fewer topic partitions + * + * @param assignment: given assignment for balance check + * @param subscriptions: topic subscriptions of each consumer + * @param partitionsPerTopic: number of partitions per topic + */ + private void verifyBalance(Map> assignment, Map> subscriptions, Map partitionsPerTopic) { + if (isFullyBalanced(assignment)) + return; + + // an ascending sorted set of consumer based on how many topic partitions are assigned to them in the given assignment + TreeSet consumers = new TreeSet(new StickyAssignor.SubscriptionComparator(assignment)); + consumers.addAll(assignment.keySet()); + + // all possible assignments + final HashMap> allSubscriptions = new HashMap<>(); + for (Map.Entry> entry: subscriptions.entrySet()) { + String consumer = entry.getKey(); + Set topicPartitions = new HashSet<>(); + allSubscriptions.put(consumer, topicPartitions); + for (String topic: entry.getValue()) { + for (int i = 0; i < partitionsPerTopic.get(topic); i++) { + TopicPartition topicPartition = new TopicPartition(topic, i); + topicPartitions.add(topicPartition); + } + } + } + + // create a mapping from partitions to the consumer assigned to them + final HashMap allPartitions = new HashMap<>(); + for (Map.Entry> entry: assignment.entrySet()) { + for (TopicPartition topicPartition: entry.getValue()) { + assertTrue(topicPartition + " is assigned to more than one consumer.", !allPartitions.containsKey(topicPartition)); + allPartitions.put(topicPartition, entry.getKey()); + } + } + + // starting from the consumer with fewest assignments make sure there is no topic partition that could be + // assigned to this consumer and it is not (because it is assigned to a consumer with more topic partitions) + for (String consumer: consumers) { + for (TopicPartition topicPartition: allSubscriptions.get(consumer)) { + if (!assignment.get(consumer).contains(topicPartition)) { + String otherConsumer = allPartitions.get(topicPartition); + assertNotNull(otherConsumer); + assertTrue(topicPartition + " can be assigned to another consumer for a more balanced assignment", assignment.get(consumer).size() >= assignment.get(otherConsumer).size() - 1); + } + } + } + } +} diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 4a49833cb762..8ff4da0916f9 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -399,7 +399,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(0, consumer0.assignment().size) - val pattern1 = Pattern.compile(".*o.*") // only 'topic' and 'foo' match this + val pattern1 = Pattern.compile(".*o.*") // only 'topic' and 'foo' match this consumer0.subscribe(pattern1, new TestConsumerReassignmentListener) consumer0.poll(50) @@ -883,6 +883,73 @@ class PlaintextConsumerTest extends BaseConsumerTest { } } + /** + * This test runs the following scenario to verify sticky assignor behavior. + * Topics: topic0 (0), topic1 (0, 1), topic2 (0, 1, 2) (numbers indicate partitions of each topic) + * Consumers: + * - C0: subscribed to topic0 + * - C1: subscribed to topic0 and topic1 + * - C2: subscribed to topic0, topic1, and topic2 + * Expected initial assignment: + * - C0: topic0-0 + * - C1: topic1-0, topic1-1 + * - C2: topic2-0, topic2-1, topic2-2 + * Then C0 unsubscribes from topic0. + * Expected new assignment: + * - C1: topic1-0, topic1-1, topic0-0 + * - C2: topic2-0, topic2-1, topic2-2 + */ + @Test + def testMultiConsumerStickyAssignment() { + this.consumers.clear() + this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "sticky-group") + this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classOf[StickyAssignor].getName) + + // create one new topics + val topic0 = "topic0" + val subscriptions0 = createTopicAndSendRecords(topic0, 1, 100) + val tp00 = new TopicPartition(topic0, 0) + + val (_, consumerPollers) = createConsumerGroupAndWaitForAssignment(1, List(topic0), subscriptions0) + validateGroupAssignment(consumerPollers, subscriptions0, s"Did not get valid initial assignment for partitions ${subscriptions0.asJava}") + + // create another new topics + val topic1 = "topic1" + val subscriptions1 = createTopicAndSendRecords(topic1, 2, 100) ++ subscriptions0 + val tp10 = new TopicPartition(topic1, 0) + val tp11 = new TopicPartition(topic1, 1) + // add one more consumer and validate re-assignment + addConsumersToGroupAndWaitForGroupAssignment(1, consumers, consumerPollers, List(topic0, topic1), subscriptions1) + + // create another new topics + val topic2 = "topic2" + val subscriptions2 = createTopicAndSendRecords(topic2, 3, 100) ++ subscriptions1 + val tp20 = new TopicPartition(topic2, 0) + val tp21 = new TopicPartition(topic2, 1) + val tp22 = new TopicPartition(topic2, 2) + // add one more consumer and validate re-assignment + addConsumersToGroupAndWaitForGroupAssignment(1, consumers, consumerPollers, List(topic0, topic1, topic2), subscriptions2) + + val expectedAssignments = Set(Set(tp00), Set(tp10, tp11), Set(tp20, tp21, tp22)) + var actualAssignmnets = Set[Set[TopicPartition]]() + consumerPollers.foreach { cp => actualAssignmnets = actualAssignmnets + cp.consumerAssignment() } + assertEquals(expectedAssignments, actualAssignmnets) + + consumerPollers.head.subscribe(List()) + val removedConsumerPoller = consumerPollers.remove(0) + val expectedNewAssignments = Set(Set(tp10, tp11, tp00), Set(tp20, tp21, tp22)) + + var actualNewAssignments = scala.collection.mutable.Set[Set[TopicPartition]]() + TestUtils.waitUntilTrue(() => { + actualNewAssignments.clear + consumerPollers.foreach { cp => actualNewAssignments = actualNewAssignments + cp.consumerAssignment() } + expectedNewAssignments == actualNewAssignments + }, s"Expected assignments ${expectedNewAssignments.asJava} but actually got ${actualNewAssignments.asJava}") + + removedConsumerPoller.shutdown() + consumerPollers.foreach(_.shutdown()) + } + /** * This test re-uses BaseConsumerTest's consumers. * As a result, it is testing the default assignment strategy set by BaseConsumerTest From 1a6cb485719ae80012a64ade3f03daaf8fe141ae Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Thu, 13 Apr 2017 10:53:16 -0700 Subject: [PATCH 2/9] Fixed the newly identified findBugs issues --- .../clients/consumer/StickyAssignor.java | 97 ++++++++++++------- .../clients/consumer/StickyAssignorTest.java | 26 ++--- 2 files changed, 75 insertions(+), 48 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java index b43904a59c1c..f55f4af1d7ad 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; +import java.io.Serializable; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Comparator; @@ -125,25 +126,22 @@ public Map> assign(Map partitionsP final HashMap> consumer2AllPotentialPartitions = new HashMap<>(); // initialize partition2AllPotentialConsumers and consumer2AllPotentialPartitions in the following two for loops - Set topics = partitionsPerTopic.keySet(); - for (String topic: topics) { - int partitions = partitionsPerTopic.get(topic); - for (int i = 0; i < partitions; ++i) - partition2AllPotentialConsumers.put(new TopicPartition(topic, i), new ArrayList()); + for (Entry entry: partitionsPerTopic.entrySet()) { + for (int i = 0; i < entry.getValue(); ++i) + partition2AllPotentialConsumers.put(new TopicPartition(entry.getKey(), i), new ArrayList()); } - Set consumers = subscriptions.keySet(); - for (String consumer: consumers) { + for (Entry> entry: subscriptions.entrySet()) { + String consumer = entry.getKey(); consumer2AllPotentialPartitions.put(consumer, new ArrayList()); - List consumerTopics = subscriptions.get(consumer); - for (String topic: consumerTopics) { - int partitions = partitionsPerTopic.get(topic); - for (int i = 0; i < partitions; ++i) { + for (String topic: entry.getValue()) { + for (int i = 0; i < partitionsPerTopic.get(topic); ++i) { TopicPartition topicPartition = new TopicPartition(topic, i); consumer2AllPotentialPartitions.get(consumer).add(topicPartition); partition2AllPotentialConsumers.get(topicPartition).add(consumer); } } + // add this consumer to currentAssignment (with an empty topic partition assignment) if it does not already exist if (!currentAssignment.containsKey(consumer)) currentAssignment.put(consumer, new ArrayList()); @@ -151,9 +149,9 @@ public Map> assign(Map partitionsP // a mapping of partition to current consumer HashMap currentPartitionConsumer = new HashMap<>(); - for (String consumer: currentAssignment.keySet()) - for (TopicPartition topicPartition: currentAssignment.get(consumer)) - currentPartitionConsumer.put(topicPartition, consumer); + for (Map.Entry> entry: currentAssignment.entrySet()) + for (TopicPartition topicPartition: entry.getValue()) + currentPartitionConsumer.put(topicPartition, entry.getKey()); // an ascending sorted set of topic partitions based on how many consumers can potentially use them TreeSet sortedAllPartitions = new TreeSet<>(new PartitionComparator(partition2AllPotentialConsumers)); @@ -289,14 +287,20 @@ protected boolean isBalanced(TreeSet sortedCurrentSubscriptions, Map> assignment) { int score = 0; - Set consumers = assignment.keySet(); - Set consumersCopy = new HashSet<>(consumers); - for (String consumer: consumers) { - consumersCopy.remove(consumer); - int consumerAssignmentSize = assignment.get(consumer).size(); - for (String otherKey: consumersCopy) - score += Math.abs(consumerAssignmentSize - assignment.get(otherKey).size()); + + Map consumer2AssignmentSize = new HashMap<>(); + for (Entry> entry: assignment.entrySet()) + consumer2AssignmentSize.put(entry.getKey(), entry.getValue().size()); + + Iterator> it = consumer2AssignmentSize.entrySet().iterator(); + while (it.hasNext()) { + Entry entry = it.next(); + int consumerAssignmentSize = entry.getValue(); + it.remove(); + for (Entry otherEntry: consumer2AssignmentSize.entrySet()) + score += Math.abs(consumerAssignmentSize - otherEntry.getValue()); } + return score; } @@ -396,8 +400,35 @@ private void balance(TreeSet sortedAllPartitions, TreeSet> preBalanceAssignment = deepCopy(currentAssignment); HashMap preBalancePartitionConsumers = deepCopy(currentPartitionConsumer); - // repeat reassignment until no partition can be moved to improve the balance + reassignmentPerformed = performReassignments(sortedAllPartitions, sortedCurrentSubscriptions, + consumer2AllPotentialPartitions, partition2AllPotentialConsumers, currentPartitionConsumer); + + // if we are not preserving existing assignments and we have made changes to the current assignment + // make sure we are getting a more balanced assignment; otherwise, revert to previous assignment + if (!initializing && reassignmentPerformed && getBalanceScore(currentAssignment) >= getBalanceScore(preBalanceAssignment)) { + deepCopy(preBalanceAssignment, currentAssignment); + currentPartitionConsumer.clear(); + currentPartitionConsumer.putAll(preBalancePartitionConsumers); + } + + // add the fixed assignments (those that could not change) back + for (Entry> entry: fixedAssignments.entrySet()) { + String consumer = entry.getKey(); + currentAssignment.put(consumer, entry.getValue()); + sortedCurrentSubscriptions.add(consumer); + } + + fixedAssignments.clear(); + } + + private boolean performReassignments(TreeSet sortedAllPartitions, TreeSet sortedCurrentSubscriptions, + HashMap> consumer2AllPotentialPartitions, + HashMap> partition2AllPotentialConsumers, + HashMap currentPartitionConsumer) { + boolean reassignmentPerformed = false; boolean modified = true; + + // repeat reassignment until no partition can be moved to improve the balance while (modified) { modified = false; // reassign all reassignable partitions (starting from the partition with least potential consumers and if needed) @@ -429,22 +460,11 @@ private void balance(TreeSet sortedAllPartitions, TreeSet= getBalanceScore(preBalanceAssignment)) { - deepCopy(preBalanceAssignment, currentAssignment); - currentPartitionConsumer = preBalancePartitionConsumers; - } - - // add the fixed assignments (those that could not change) back - for (String consumer: fixedAssignments.keySet()) { - currentAssignment.put(consumer, fixedAssignments.get(consumer)); - sortedCurrentSubscriptions.add(consumer); - } - fixedAssignments.clear(); + return reassignmentPerformed; } - private static class PartitionComparator implements Comparator { + @SuppressWarnings("serial") + private static class PartitionComparator implements Comparator, Serializable { private Map> map; PartitionComparator(Map> map) { @@ -462,7 +482,9 @@ public int compare(TopicPartition o1, TopicPartition o2) { return ret; } } - protected static class SubscriptionComparator implements Comparator { + + @SuppressWarnings("serial") + protected static class SubscriptionComparator implements Comparator, Serializable { private Map> map; SubscriptionComparator(Map> map) { @@ -477,4 +499,5 @@ public int compare(String o1, String o2) { return ret; } } + } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java index 8196089b7629..2d0626757412 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java @@ -1,14 +1,18 @@ -/** - * 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. +/* + * 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.clients.consumer; From 9beea2ff6da4270a7bdd1ac08f8b73548982d919 Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Wed, 19 Apr 2017 15:34:16 -0700 Subject: [PATCH 3/9] Improved stickiness for when there is uniform topic subscription across consumers --- .../clients/consumer/StickyAssignor.java | 120 +++++++++++++++--- .../clients/consumer/StickyAssignorTest.java | 21 ++- .../kafka/api/PlaintextConsumerTest.scala | 89 ++++++------- 3 files changed, 160 insertions(+), 70 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java index f55f4af1d7ad..c8a3fc3bf9ba 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java @@ -24,6 +24,7 @@ import java.io.Serializable; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collection; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -153,12 +154,10 @@ public Map> assign(Map partitionsP for (TopicPartition topicPartition: entry.getValue()) currentPartitionConsumer.put(topicPartition, entry.getKey()); - // an ascending sorted set of topic partitions based on how many consumers can potentially use them - TreeSet sortedAllPartitions = new TreeSet<>(new PartitionComparator(partition2AllPotentialConsumers)); - sortedAllPartitions.addAll(partition2AllPotentialConsumers.keySet()); + List sortedPartitions = sortPartitions(oldAssignment.isEmpty(), partition2AllPotentialConsumers, consumer2AllPotentialPartitions); // all partitions that need to be assigned (initially set to all partitions but adjusted in the following loop) - TreeSet unassignedPartitions = new TreeSet<>(sortedAllPartitions); + List unassignedPartitions = new ArrayList<>(sortedPartitions); for (Iterator>> it = currentAssignment.entrySet().iterator(); it.hasNext();) { Map.Entry> entry = it.next(); if (!subscriptions.containsKey(entry.getKey())) { @@ -189,7 +188,7 @@ public Map> assign(Map partitionsP TreeSet sortedCurrentSubscriptions = new TreeSet<>(new SubscriptionComparator(currentAssignment)); sortedCurrentSubscriptions.addAll(currentAssignment.keySet()); - balance(sortedAllPartitions, unassignedPartitions, sortedCurrentSubscriptions, consumer2AllPotentialPartitions, + balance(sortedPartitions, unassignedPartitions, sortedCurrentSubscriptions, consumer2AllPotentialPartitions, partition2AllPotentialConsumers, oldAssignment, currentPartitionConsumer); return currentAssignment; } @@ -304,6 +303,97 @@ protected int getBalanceScore(Map> assignment) { return score; } + /** + * Sort valid partitions so they are processed in the potential reassignment phase in the proper order + * that causes minimal partition movement among consumers (hence honoring maximal stickiness) + * + * @param isFreshAssignment whether this is a new assignment, or a reassignment of an existing one + * @param partition2AllPotentialConsumers a mapping of partitions to their potential consumers + * @param consumer2AllPotentialPartitions a mapping of consumers to potential partitions they can consumer from + * @return sorted list of valid partitions + */ + private List sortPartitions(boolean isFreshAssignment, + HashMap> partition2AllPotentialConsumers, + HashMap> consumer2AllPotentialPartitions) { + List sortedPartitions = new ArrayList<>(); + + if (!isFreshAssignment && areSubscriptionsIdentical(partition2AllPotentialConsumers, consumer2AllPotentialPartitions)) { + // if this is a reassignment and the subscriptions are identical (all consumers can consumer from all topics) + // then we just need to simply list partitions in a round robin fashion (from consumers with + // most assigned partitions to those with least) + Map> assignments = deepCopy(currentAssignment); + for (Entry> entry: assignments.entrySet()) { + List toRemove = new ArrayList<>(); + for (TopicPartition partition: entry.getValue()) + if (!partition2AllPotentialConsumers.keySet().contains(partition)) + toRemove.add(partition); + for (TopicPartition partition: toRemove) + entry.getValue().remove(partition); + } + TreeSet sortedConsumers = new TreeSet<>(new SubscriptionComparator(assignments)); + sortedConsumers.addAll(assignments.keySet()); + + while (!sortedConsumers.isEmpty()) { + String consumer = sortedConsumers.pollLast(); + List remainingPartitions = assignments.get(consumer); + if (!remainingPartitions.isEmpty()) { + sortedPartitions.add(remainingPartitions.remove(0)); + sortedConsumers.add(consumer); + } + } + + for (TopicPartition partition: partition2AllPotentialConsumers.keySet()) { + if (!sortedPartitions.contains(partition)) + sortedPartitions.add(partition); + } + + } else { + // an ascending sorted set of topic partitions based on how many consumers can potentially use them + TreeSet sortedAllPartitions = new TreeSet<>(new PartitionComparator(partition2AllPotentialConsumers)); + sortedAllPartitions.addAll(partition2AllPotentialConsumers.keySet()); + + while (!sortedAllPartitions.isEmpty()) + sortedPartitions.add(sortedAllPartitions.pollFirst()); + } + + return sortedPartitions; + } + + /** + * @param partition2AllPotentialConsumers a mapping of partitions to their potential consumers + * @param consumer2AllPotentialPartitions a mapping of consumers to potential partitions they can consumer from + * @return true if potential consumers of partitions are the same, and potential partitions consumers can + * consumer from are the same too + */ + private boolean areSubscriptionsIdentical(HashMap> partition2AllPotentialConsumers, + HashMap> consumer2AllPotentialPartitions) { + if (!hasIdenticalListElements(partition2AllPotentialConsumers.values())) + return false; + + if (!hasIdenticalListElements(consumer2AllPotentialPartitions.values())) + return false; + + return true; + } + + /** + * @param col a collection of elements that are lists themselves + * @return true if all lists in the collection have the same members; false otherwise + */ + private boolean hasIdenticalListElements(Collection> col) { + Set> set = new HashSet>(col.size()); + for (List l: col) { + final Set s = new HashSet<>(l); + if (set.isEmpty()) + set.add(s); + else { + if (set.add(s)) + return false; + } + } + return true; + } + /** * @return the consumer to which the given partition is assigned. The assignment should improve the overall balance * of the partition assignments to consumers. @@ -362,16 +452,14 @@ private boolean canParticipateInReassignment(String consumer, /** * Balance the current assignment using the data structures created in the assign(...) method above. */ - private void balance(TreeSet sortedAllPartitions, TreeSet unassignedPartitions, TreeSet sortedCurrentSubscriptions, + private void balance(List sortedPartitions, List unassignedPartitions, TreeSet sortedCurrentSubscriptions, HashMap> consumer2AllPotentialPartitions, HashMap> partition2AllPotentialConsumers, Map> oldAssignment, HashMap currentPartitionConsumer) { boolean initializing = currentAssignment.get(sortedCurrentSubscriptions.last()).isEmpty(); boolean reassignmentPerformed = false; // assign all unassigned partitions - while (!unassignedPartitions.isEmpty()) { - TopicPartition partition = unassignedPartitions.pollFirst(); - + for (TopicPartition partition: unassignedPartitions) { // skip if there is no potential consumer for the partition if (partition2AllPotentialConsumers.get(partition).isEmpty()) continue; @@ -386,7 +474,7 @@ private void balance(TreeSet sortedAllPartitions, TreeSet> fixedAssignments = new HashMap<>(); @@ -400,7 +488,7 @@ private void balance(TreeSet sortedAllPartitions, TreeSet> preBalanceAssignment = deepCopy(currentAssignment); HashMap preBalancePartitionConsumers = deepCopy(currentPartitionConsumer); - reassignmentPerformed = performReassignments(sortedAllPartitions, sortedCurrentSubscriptions, + reassignmentPerformed = performReassignments(sortedPartitions, sortedCurrentSubscriptions, consumer2AllPotentialPartitions, partition2AllPotentialConsumers, currentPartitionConsumer); // if we are not preserving existing assignments and we have made changes to the current assignment @@ -421,10 +509,10 @@ private void balance(TreeSet sortedAllPartitions, TreeSet sortedAllPartitions, TreeSet sortedCurrentSubscriptions, - HashMap> consumer2AllPotentialPartitions, - HashMap> partition2AllPotentialConsumers, - HashMap currentPartitionConsumer) { + private boolean performReassignments(List sortedPartitions, TreeSet sortedCurrentSubscriptions, + HashMap> consumer2AllPotentialPartitions, + HashMap> partition2AllPotentialConsumers, + HashMap currentPartitionConsumer) { boolean reassignmentPerformed = false; boolean modified = true; @@ -433,7 +521,7 @@ private boolean performReassignments(TreeSet sortedAllPartitions modified = false; // reassign all reassignable partitions (starting from the partition with least potential consumers and if needed) // until the full list is processed or a balance is achieved - Iterator partitionIterator = sortedAllPartitions.iterator(); + Iterator partitionIterator = sortedPartitions.iterator(); while (partitionIterator.hasNext() && !isBalanced(sortedCurrentSubscriptions, consumer2AllPotentialPartitions)) { TopicPartition partition = partitionIterator.next(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java index 2d0626757412..fcc123fdfdee 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java @@ -329,7 +329,7 @@ public void testAddRemoveTopicTwoConsumers() { subscriptions.put(consumer2, topics(topic, topic2)); Map> assignment2 = assignor.assign(partitionsPerTopic, subscriptions); assertEquals(Arrays.asList(new TopicPartition(topic, 0), new TopicPartition(topic, 2), new TopicPartition(topic2, 1)), assignment2.get(consumer1)); - assertEquals(Arrays.asList(new TopicPartition(topic, 1), new TopicPartition(topic2, 0), new TopicPartition(topic2, 2)), assignment2.get(consumer2)); + assertEquals(Arrays.asList(new TopicPartition(topic, 1), new TopicPartition(topic2, 2), new TopicPartition(topic2, 0)), assignment2.get(consumer2)); assertTrue(isFullyBalanced(assignment2)); verifyBalance(assignment, subscriptions, partitionsPerTopic); @@ -338,7 +338,7 @@ public void testAddRemoveTopicTwoConsumers() { subscriptions.put(consumer2, topics(topic2)); Map> assignment3 = assignor.assign(partitionsPerTopic, subscriptions); assertEquals(Arrays.asList(new TopicPartition(topic2, 1)), assignment3.get(consumer1)); - assertEquals(Arrays.asList(new TopicPartition(topic2, 0), new TopicPartition(topic2, 2)), assignment3.get(consumer2)); + assertEquals(Arrays.asList(new TopicPartition(topic2, 2), new TopicPartition(topic2, 0)), assignment3.get(consumer2)); assertTrue(isFullyBalanced(assignment3)); verifyBalance(assignment, subscriptions, partitionsPerTopic); } @@ -365,6 +365,23 @@ public void testReassignmentAfterOneConsumerLeaves() { verifyBalance(assignment, subscriptions, partitionsPerTopic); } + @Test + public void testReassignmentAfterOneConsumerAdded() { + Map partitionsPerTopic = new HashMap<>(); + partitionsPerTopic.put("topic", 20); + + Map> subscriptions = new HashMap<>(); + for (int i = 1; i < 10; i++) + subscriptions.put(String.format("consumer%02d", i), Collections.singletonList("topic")); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + + subscriptions.put("consumer10", Collections.singletonList("topic")); + assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyBalance(assignment, subscriptions, partitionsPerTopic); + } + @Test public void testSameSubscriptions() { Map partitionsPerTopic = new HashMap<>(); diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 8ff4da0916f9..e565ce23fe09 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -883,21 +883,19 @@ class PlaintextConsumerTest extends BaseConsumerTest { } } + def reverse(m: Map[Long, Set[TopicPartition]]) = + m.values.toSet.flatten.map(v => (v, m.keys.filter(m(_).contains(v)).head)).toMap + /** * This test runs the following scenario to verify sticky assignor behavior. - * Topics: topic0 (0), topic1 (0, 1), topic2 (0, 1, 2) (numbers indicate partitions of each topic) - * Consumers: - * - C0: subscribed to topic0 - * - C1: subscribed to topic0 and topic1 - * - C2: subscribed to topic0, topic1, and topic2 - * Expected initial assignment: - * - C0: topic0-0 - * - C1: topic1-0, topic1-1 - * - C2: topic2-0, topic2-1, topic2-2 - * Then C0 unsubscribes from topic0. + * Topics: single-topic, with random number of partitions, where #par is 10, 20, 30, 40, 50, 60, 70, 80, 90, or 100 + * Consumers: 9 consumers subscribed to the single topic + * Expected initial assignment: partitions are assigned to consumers in a round robin fashion. + * - (#par mod 9) consumers will get (#par / 9 + 1) partitions, and the rest get (#par / 9) partitions + * Then consumer #10 is added to the list (subscribing to the same single topic) * Expected new assignment: - * - C1: topic1-0, topic1-1, topic0-0 - * - C2: topic2-0, topic2-1, topic2-2 + * - (#par / 10) partition per consumer, where one partition from each of the early (#par mod 9) consumers + * will move to consumer #10, leading to a total of (#par mod 9) partition movement */ @Test def testMultiConsumerStickyAssignment() { @@ -905,49 +903,36 @@ class PlaintextConsumerTest extends BaseConsumerTest { this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "sticky-group") this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classOf[StickyAssignor].getName) - // create one new topics - val topic0 = "topic0" - val subscriptions0 = createTopicAndSendRecords(topic0, 1, 100) - val tp00 = new TopicPartition(topic0, 0) - - val (_, consumerPollers) = createConsumerGroupAndWaitForAssignment(1, List(topic0), subscriptions0) - validateGroupAssignment(consumerPollers, subscriptions0, s"Did not get valid initial assignment for partitions ${subscriptions0.asJava}") + // create one new topic + val topic = "single-topic" + val rand = 1 + scala.util.Random.nextInt(10) + val partitions = createTopicAndSendRecords(topic, rand * 10, 100) - // create another new topics - val topic1 = "topic1" - val subscriptions1 = createTopicAndSendRecords(topic1, 2, 100) ++ subscriptions0 - val tp10 = new TopicPartition(topic1, 0) - val tp11 = new TopicPartition(topic1, 1) - // add one more consumer and validate re-assignment - addConsumersToGroupAndWaitForGroupAssignment(1, consumers, consumerPollers, List(topic0, topic1), subscriptions1) + // create a group of consumers, subscribe the consumers to the single topic and start polling + // for the topic partition assignment + val (_, consumerPollers) = createConsumerGroupAndWaitForAssignment(9, List(topic), partitions) + validateGroupAssignment(consumerPollers, partitions, s"Did not get valid initial assignment for partitions ${partitions.asJava}") + val prePartition2PollerId = reverse(consumerPollers.map(poller => (poller.getId, poller.consumerAssignment())).toMap) - // create another new topics - val topic2 = "topic2" - val subscriptions2 = createTopicAndSendRecords(topic2, 3, 100) ++ subscriptions1 - val tp20 = new TopicPartition(topic2, 0) - val tp21 = new TopicPartition(topic2, 1) - val tp22 = new TopicPartition(topic2, 2) // add one more consumer and validate re-assignment - addConsumersToGroupAndWaitForGroupAssignment(1, consumers, consumerPollers, List(topic0, topic1, topic2), subscriptions2) - - val expectedAssignments = Set(Set(tp00), Set(tp10, tp11), Set(tp20, tp21, tp22)) - var actualAssignmnets = Set[Set[TopicPartition]]() - consumerPollers.foreach { cp => actualAssignmnets = actualAssignmnets + cp.consumerAssignment() } - assertEquals(expectedAssignments, actualAssignmnets) - - consumerPollers.head.subscribe(List()) - val removedConsumerPoller = consumerPollers.remove(0) - val expectedNewAssignments = Set(Set(tp10, tp11, tp00), Set(tp20, tp21, tp22)) - - var actualNewAssignments = scala.collection.mutable.Set[Set[TopicPartition]]() - TestUtils.waitUntilTrue(() => { - actualNewAssignments.clear - consumerPollers.foreach { cp => actualNewAssignments = actualNewAssignments + cp.consumerAssignment() } - expectedNewAssignments == actualNewAssignments - }, s"Expected assignments ${expectedNewAssignments.asJava} but actually got ${actualNewAssignments.asJava}") + addConsumersToGroupAndWaitForGroupAssignment(1, consumers, consumerPollers, List(topic), partitions) + + val postPartition2PollerId = reverse(consumerPollers.map(poller => (poller.getId, poller.consumerAssignment())).toMap) + val keys = prePartition2PollerId.keySet.union(postPartition2PollerId.keySet) + var changes = 0 + keys.foreach { key => + val preVal = prePartition2PollerId.get(key) + val postVal = postPartition2PollerId.get(key) + if (preVal.nonEmpty && postVal.nonEmpty) { + if (preVal.get != postVal.get) + changes += 1 + } else + changes += 1 + } - removedConsumerPoller.shutdown() consumerPollers.foreach(_.shutdown()) + + assertEquals("Expected only two topic partitions that have switched to other consumers.", rand, changes) } /** @@ -1544,8 +1529,8 @@ class PlaintextConsumerTest extends BaseConsumerTest { * Subscribes consumer 'consumer' to a given list of topics 'topicsToSubscribe', creates * consumer poller and starts polling. * Assumes that the consumer is not subscribed to any topics yet - * - * @param consumer consumer + * + * @param consumer consumer * @param topicsToSubscribe topics that this consumer will subscribe to * @return consumer poller for the given consumer */ From 6fa95a51b50266a6756e06fa54ca07043ee06f41 Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Wed, 19 Apr 2017 15:34:16 -0700 Subject: [PATCH 4/9] Improved stickiness for when there is uniform topic subscription across consumers --- .../clients/consumer/StickyAssignorTest.java | 27 ++++++++++++++----- 1 file changed, 21 insertions(+), 6 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java index fcc123fdfdee..4c81d20a23da 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java @@ -318,29 +318,44 @@ public void testAddRemoveTopicTwoConsumers() { subscriptions.put(consumer2, topics(topic)); Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - assertEquals(Arrays.asList(new TopicPartition(topic, 0), new TopicPartition(topic, 2)), assignment.get(consumer1)); - assertEquals(Arrays.asList(new TopicPartition(topic, 1)), assignment.get(consumer2)); + // verify balance assertTrue(isFullyBalanced(assignment)); verifyBalance(assignment, subscriptions, partitionsPerTopic); + // verify stickiness + List consumer1Assignment1 = assignment.get(consumer1); + List consumer2Assignment1 = assignment.get(consumer2); + assertTrue((consumer1Assignment1.size() == 1 && consumer2Assignment1.size() == 2) || + (consumer1Assignment1.size() == 2 && consumer2Assignment1.size() == 1)); String topic2 = "topic2"; partitionsPerTopic.put(topic2, 3); subscriptions.put(consumer1, topics(topic, topic2)); subscriptions.put(consumer2, topics(topic, topic2)); Map> assignment2 = assignor.assign(partitionsPerTopic, subscriptions); - assertEquals(Arrays.asList(new TopicPartition(topic, 0), new TopicPartition(topic, 2), new TopicPartition(topic2, 1)), assignment2.get(consumer1)); - assertEquals(Arrays.asList(new TopicPartition(topic, 1), new TopicPartition(topic2, 2), new TopicPartition(topic2, 0)), assignment2.get(consumer2)); + // verify balance assertTrue(isFullyBalanced(assignment2)); verifyBalance(assignment, subscriptions, partitionsPerTopic); + // verify stickiness + List consumer1Assignment2 = assignment2.get(consumer1); + List consumer2Assignment2 = assignment2.get(consumer2); + assertTrue(consumer1Assignment2.size() == 3 && consumer2Assignment2.size() == 3); + assertTrue(consumer1Assignment2.containsAll(consumer1Assignment1)); + assertTrue(consumer2Assignment2.containsAll(consumer2Assignment1)); partitionsPerTopic.remove(topic); subscriptions.put(consumer1, topics(topic2)); subscriptions.put(consumer2, topics(topic2)); Map> assignment3 = assignor.assign(partitionsPerTopic, subscriptions); - assertEquals(Arrays.asList(new TopicPartition(topic2, 1)), assignment3.get(consumer1)); - assertEquals(Arrays.asList(new TopicPartition(topic2, 2), new TopicPartition(topic2, 0)), assignment3.get(consumer2)); + // verify balance assertTrue(isFullyBalanced(assignment3)); verifyBalance(assignment, subscriptions, partitionsPerTopic); + // verify stickiness + List consumer1Assignment3 = assignment3.get(consumer1); + List consumer2Assignment3 = assignment3.get(consumer2); + assertTrue((consumer1Assignment3.size() == 1 && consumer2Assignment3.size() == 2) || + (consumer1Assignment3.size() == 2 && consumer2Assignment3.size() == 1)); + assertTrue(consumer1Assignment2.containsAll(consumer1Assignment3)); + assertTrue(consumer2Assignment2.containsAll(consumer2Assignment3)); } @Test From 0a582ec89c563299820528296ccb9951a0c9a97c Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Thu, 27 Apr 2017 16:15:14 -0700 Subject: [PATCH 5/9] Modify the approach to preserve assignments in case of a leader change Instead of having all group members keep a copy of all member assignments, leverage the `userData` field in `Subscription` object of each member, and have each member announce their previous assignment to the leader prior to leader performing the assignment. --- .../clients/consumer/StickyAssignor.java | 207 ++++++++++++------ .../internals/AbstractPartitionAssignor.java | 9 +- .../consumer/internals/ConsumerProtocol.java | 36 +++ 3 files changed, 176 insertions(+), 76 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java index c8a3fc3bf9ba..687b813d0155 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java @@ -16,11 +16,6 @@ */ package org.apache.kafka.clients.consumer; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.io.Serializable; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -36,6 +31,7 @@ import java.util.TreeSet; import org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor; +import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; import org.apache.kafka.common.TopicPartition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,7 +39,9 @@ /** * The sticky assignor serves two purposes. First, it guarantees an assignment that is as balanced as possible, meaning either: * - the numbers of topic partitions assigned to consumers differ by at most one; or - * - each consumer that has 2+ fewer topic partitions than some other consumer cannot get any of those topic partitions transfered to it. + * - each consumer that has 2+ fewer topic partitions than some other consumer cannot get any of those topic partitions transferred to it. + * Second, it preserved as many existing assignment as possible when a reassignment occurs. This helps in saving some of the + * overhead processing when topic partitions move from one consumer to another. * * Starting fresh it would work by distributing the partitions over consumers as evenly as possible. Even though this may sound similar to * how round robin assignor works, the second example below shows that it is not. @@ -52,56 +50,132 @@ * 2. topic partitions stay with their previously assigned consumers as much as possible. * Of course, the first goal above takes precedence over the second one. * - * Example 1. Suppose there are three consumers C0, C1, C2, four topics t0, t1, t2, t3, and each topic has 2 partitions, - * resulting in partitions t0p0, t0p1, t1p0, t1p1, t2p0, t2p1, t3p0, t3p1. Each consumer is subscribed to all three topics. + * Example 1. Suppose there are three consumers C0, C1, C2, + * four topics t0, t1, t2, t3, and each topic has 2 partitions, + * resulting in partitions t0p0, t0p1, t1p0, t1p1, t2p0, + * t2p1, t3p0, t3p1. Each consumer is subscribed to all three topics. * * The assignment with both sticky and round robin assignors will be: - * C0: [t0p0, t1p1, t3p0] - * C1: [t0p1, t2p0, t3p1] - * C2: [t1p0, t2p1] + *

    + *
  • C0: [t0p0, t1p1, t3p0]
  • + *
  • C1: [t0p1, t2p0, t3p1]
  • + *
  • C2: [t1p0, t2p1]
  • + *
* - * Now, let's assume C1 is removed and a reassignment is about to happen. The round robin assignor would produce: - * C0: [t0p0, t1p0, t2p0, t3p0] - * C2: [t0p1, t1p1, t2p1, t3p1] + * Now, let's assume C1 is removed and a reassignment is about to happen. The round robin assignor would produce: + *
    + *
  • C0: [t0p0, t1p0, t2p0, t3p0]
  • + *
  • C2: [t0p1, t1p1, t2p1, t3p1]
  • + *
* * while the sticky assignor would result in: - * C0 [t0p0, t1p1, t3p0, t2p0] - * C2 [t1p0, t2p1, t0p1, t3p1] + *
    + *
  • C0 [t0p0, t1p1, t3p0, t2p0]
  • + *
  • C2 [t1p0, t2p1, t0p1, t3p1]
  • + *
* preserving all the previous assignments (unlike the round robin assignor). * - * Example 2. There are three consumers C0, C1, C2, and three topics t0, t1, t2, with 1, 2, and 3 partitions respectively. - * Therefore, the partitions are t0p0, t1p0, t1p1, t2p0, t2p1, t2p2. - * C0 is subscribed to t0; C1 is subscribed to t0, t1; and C2 is subscribed to t0, t1, t2. + * Example 2. There are three consumers C0, C1, C2, + * and three topics t0, t1, t2, with 1, 2, and 3 partitions respectively. + * Therefore, the partitions are t0p0, t1p0, t1p1, t2p0, + * t2p1, t2p2. C0 is subscribed to t0; C1 is subscribed to + * t0, t1; and C2 is subscribed to t0, t1, t2. * * The round robin assignor would come up with the following assignment: - * C0 [t0p0] - * C1 [t1p0] - * C2 [t1p1, t2p0, t2p1, t2p2] + *
    + *
  • C0 [t0p0]
  • + *
  • C1 [t1p0]
  • + *
  • C2 [t1p1, t2p0, t2p1, t2p2]
  • + *
* * which is not as balanced as the assignment suggested by sticky assignor: - * C0 [t0p0] - * C1 [t1p0, t1p1] - * C2 [t2p0, t2p1, t2p2] + *
    + *
  • C0 [t0p0]
  • + *
  • C1 [t1p0, t1p1]
  • + *
  • C2 [t2p0, t2p1, t2p2]
  • + *
* - * Now, if consumer C0 is removed, these two assignors would produce the following assignments. + * Now, if consumer C0 is removed, these two assignors would produce the following assignments. * Round Robin (preserves 3 partition assignments): - * C1 [t0p0, t1p1] - * C2 [t1p0, t2p0, t2p1, t2p2] + *
    + *
  • C1 [t0p0, t1p1]
  • + *
  • C2 [t1p0, t2p0, t2p1, t2p2]
  • + *
* * Sticky (preserves 5 partition assignments): - * C1 [t1p0, t1p1, t0p0] - * C2 [t2p0, t2p1, t2p2] + *
    + *
  • C1 [t1p0, t1p1, t0p0]
  • + *
  • C2 [t2p0, t2p1, t2p2]
  • + *
+ * + *

Impact on ConsumerRebalanceListener

+ * The sticky assignment strategy can provide some optimization to those consumers that have some partition cleanup code + * in their onPartitionsRevoked() callback listeners. The cleanup code is placed in that callback listener + * because the consumer has no assumption or hope of preserving any of its assigned partitions after a rebalance when it + * is using range or round robin assignor. The listener code would look like this: + * + * class TheOldRebalanceListener implements ConsumerRebalanceListener { + * + * void onPartitionsRevoked(Collection partitions) { + * for (TopicPartition partition: partitions) { + * commitOffsets(partition); + * cleanupState(partition); + * } + * } + * + * void onPartitionsAssigned(Collection partitions) { + * for (TopicPartition partition: partitions) { + * initializeState(partition); + * initializeOffset(partition); + * } + * } + * } + * + * + * As mentioned above, one advantage of the sticky assignor is that, in general, it reduces the number of partitions that + * actually move from one consumer to another during a reassignment. Therefore, it allows consumers to do their cleanup + * more efficiently. Of course, they still can perform the partition cleanup in the onPartitionsRevoked() + * listener, but they can be more efficient and make a note of their partitions before and after the rebalance, and do the + * cleanup after the rebalance only on the partitions they have lost (which is normally not a lot). The code snippet below + * clarifies this point: + * + * class TheNewRebalanceListener implements ConsumerRebalanceListener { + * Collection lastAssignment = Collections.emptyList(); + * + * void onPartitionsRevoked(Collection partitions) { + * for (TopicPartition partition: partitions) + * commitOffsets(partition); + * } + * + * void onPartitionsAssigned(Collection assignment) { + * for (TopicPartition partition: difference(lastAssignment, assignment)) + * cleanupState(partition); + * + * for (TopicPartition partition: difference(assignment, lastAssignment)) + * initializeState(partition); + * + * for (TopicPartition partition: assignment) + * initializeOffset(partition); + * + * this.lastAssignment = assignment; + * } + * } + * + * + * Any consumer that uses sticky assignment can leverage this listener like this: + * consumer.subscribe(topics, new TheNewRebalanceListener()); * */ public class StickyAssignor extends AbstractPartitionAssignor { private static final Logger log = LoggerFactory.getLogger(StickyAssignor.class); protected Map> currentAssignment = new HashMap<>(); + private List memberAssignment = null; private void deepCopy(Map> source, Map> dest) { dest.clear(); for (Entry> entry: source.entrySet()) - dest.put(entry.getKey(), new ArrayList(entry.getValue())); + dest.put(entry.getKey(), new ArrayList<>(entry.getValue())); } private Map> deepCopy(Map> assignment) { @@ -110,14 +184,9 @@ private Map> deepCopy(Map deepCopy(HashMap source) { - HashMap copy = new HashMap<>(); - copy.putAll(source); - return copy; - } - public Map> assign(Map partitionsPerTopic, - Map> subscriptions) { + Map> subscriptions) { + prepopulateCurrentAssignments(); // make a deep copy of currentAssignment Map> oldAssignment = deepCopy(currentAssignment); @@ -193,33 +262,29 @@ public Map> assign(Map partitionsP return currentAssignment; } + private void prepopulateCurrentAssignments() { + Map subscriptions = getSubscriptions(); + if (subscriptions == null) + return; - @Override - protected ByteBuffer getUserData() { - try { - ByteArrayOutputStream bos = new ByteArrayOutputStream(2048); - ObjectOutputStream oos = new ObjectOutputStream(bos); - oos.writeObject(currentAssignment); - return ByteBuffer.wrap(bos.toByteArray()); - } catch (Exception ioex) { - log.error("Failed to serialize currentAssignment", ioex); - return super.getUserData(); + for (Map.Entry subscriptionEntry : subscriptions.entrySet()) { + ByteBuffer userData = subscriptionEntry.getValue().userData(); + if (userData != null && userData.hasRemaining()) + currentAssignment.put(subscriptionEntry.getKey(), ConsumerProtocol.deserializeTopicPartitionAssignment(userData)); } } - @SuppressWarnings("unchecked") @Override public void onAssignment(Assignment assignment) { - try { - byte[] array = new byte[assignment.userData().remaining()]; - assignment.userData().get(array); - ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(array)); - currentAssignment = (Map>) ois.readObject(); - log.debug("Got currentAssignment = " + currentAssignment); - } catch (IOException | ClassNotFoundException exc) { - log.error("Failed to deserialize assignment userdata into currentAssignment", exc); - currentAssignment.clear(); - } + memberAssignment = assignment.partitions(); + } + + @Override + public Subscription subscription(Set topics) { + if (memberAssignment == null) + return new Subscription(new ArrayList<>(topics)); + + return new Subscription(new ArrayList<>(topics), ConsumerProtocol.serializeTopicPartitionAssignment(memberAssignment)); } @Override @@ -377,19 +442,17 @@ private boolean areSubscriptionsIdentical(HashMap> } /** - * @param col a collection of elements that are lists themselves + * @param col a collection of elements of type list * @return true if all lists in the collection have the same members; false otherwise */ private boolean hasIdenticalListElements(Collection> col) { - Set> set = new HashSet>(col.size()); - for (List l: col) { - final Set s = new HashSet<>(l); - if (set.isEmpty()) - set.add(s); - else { - if (set.add(s)) - return false; - } + Iterator> it = col.iterator(); + List cur = it.next(); + while (it.hasNext()) { + List next = it.next(); + if (!(cur.containsAll(next) && next.containsAll(cur))) + return false; + cur = next; } return true; } @@ -415,7 +478,7 @@ private String assignPartition(TopicPartition partition, TreeSet sortedC /** * Remove partition from partition assignments of the given consumer */ - private void deassignPartition(TopicPartition partition, String consumer, TreeSet sortedCurrentSubscriptions, HashMap currentPartitionConsumer) { + private void unassignPartition(TopicPartition partition, String consumer, TreeSet sortedCurrentSubscriptions, HashMap currentPartitionConsumer) { sortedCurrentSubscriptions.remove(consumer); currentAssignment.get(consumer).remove(partition); currentPartitionConsumer.remove(partition); @@ -486,7 +549,7 @@ private void balance(List sortedPartitions, List // create a deep copy of the current assignment so we can revert to it if we do not get a more balanced assignment later Map> preBalanceAssignment = deepCopy(currentAssignment); - HashMap preBalancePartitionConsumers = deepCopy(currentPartitionConsumer); + HashMap preBalancePartitionConsumers = new HashMap<>(currentPartitionConsumer); reassignmentPerformed = performReassignments(sortedPartitions, sortedCurrentSubscriptions, consumer2AllPotentialPartitions, partition2AllPotentialConsumers, currentPartitionConsumer); @@ -538,7 +601,7 @@ private boolean performReassignments(List sortedPartitions, Tree for (String otherConsumer: partition2AllPotentialConsumers.get(partition)) if (currentAssignment.get(consumer).size() > currentAssignment.get(otherConsumer).size() + 1) { // de-assign partition from its current consumer - deassignPartition(partition, consumer, sortedCurrentSubscriptions, currentPartitionConsumer); + unassignPartition(partition, consumer, sortedCurrentSubscriptions, currentPartitionConsumer); // reassign the partition to an eligible consumer with fewest assignments assignPartition(partition, sortedCurrentSubscriptions, consumer2AllPotentialPartitions, currentPartitionConsumer); reassignmentPerformed = true; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java index 7ae7fa8455db..2fcd3ff235b2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java @@ -21,7 +21,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; @@ -35,6 +34,7 @@ */ public abstract class AbstractPartitionAssignor implements PartitionAssignor { private static final Logger log = LoggerFactory.getLogger(AbstractPartitionAssignor.class); + private Map subscriptions = null; /** * Perform the group assignment given the partition counts and member subscriptions @@ -53,6 +53,7 @@ public Subscription subscription(Set topics) { @Override public Map assign(Cluster metadata, Map subscriptions) { + this.subscriptions = new HashMap<>(subscriptions); Set allSubscribedTopics = new HashSet<>(); Map> topicSubscriptions = new HashMap<>(); for (Map.Entry subscriptionEntry : subscriptions.entrySet()) { @@ -75,12 +76,12 @@ public Map assign(Cluster metadata, Map assignments = new HashMap<>(); for (Map.Entry> assignmentEntry : rawAssignments.entrySet()) - assignments.put(assignmentEntry.getKey(), new Assignment(assignmentEntry.getValue(), getUserData())); + assignments.put(assignmentEntry.getKey(), new Assignment(assignmentEntry.getValue())); return assignments; } - protected ByteBuffer getUserData() { - return ByteBuffer.wrap(new byte[0]); + protected Map getSubscriptions() { + return subscriptions; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java index 392e27289d20..07dd9b87ecab 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java @@ -78,6 +78,8 @@ public class ConsumerProtocol { public static final Schema ASSIGNMENT_V0 = new Schema( new Field(TOPIC_PARTITIONS_KEY_NAME, new ArrayOf(TOPIC_ASSIGNMENT_V0)), new Field(USER_DATA_KEY_NAME, Type.NULLABLE_BYTES)); + public static final Schema TOPIC_PARTITION_ASSIGNMENT_V0 = new Schema( + new Field(TOPIC_PARTITIONS_KEY_NAME, new ArrayOf(TOPIC_ASSIGNMENT_V0))); public static ByteBuffer serializeSubscription(PartitionAssignor.Subscription subscription) { Struct struct = new Struct(SUBSCRIPTION_V0); @@ -138,6 +140,40 @@ public static ByteBuffer serializeAssignment(PartitionAssignor.Assignment assign return buffer; } + public static ByteBuffer serializeTopicPartitionAssignment(List partitions) { + Struct struct = new Struct(TOPIC_PARTITION_ASSIGNMENT_V0); + List topicAssignments = new ArrayList<>(); + for (Map.Entry> topicEntry : asMap(partitions).entrySet()) { + Struct topicAssignment = new Struct(TOPIC_ASSIGNMENT_V0); + topicAssignment.set(TOPIC_KEY_NAME, topicEntry.getKey()); + topicAssignment.set(PARTITIONS_KEY_NAME, topicEntry.getValue().toArray()); + topicAssignments.add(topicAssignment); + } + struct.set(TOPIC_PARTITIONS_KEY_NAME, topicAssignments.toArray()); + ByteBuffer buffer = ByteBuffer.allocate(CONSUMER_PROTOCOL_HEADER_V0.sizeOf() + TOPIC_PARTITION_ASSIGNMENT_V0.sizeOf(struct)); + CONSUMER_PROTOCOL_HEADER_V0.writeTo(buffer); + TOPIC_PARTITION_ASSIGNMENT_V0.write(buffer, struct); + buffer.flip(); + return buffer; + } + + public static List deserializeTopicPartitionAssignment(ByteBuffer buffer) { + Struct header = CONSUMER_PROTOCOL_HEADER_SCHEMA.read(buffer); + Short version = header.getShort(VERSION_KEY_NAME); + checkVersionCompatibility(version); + Struct struct = TOPIC_PARTITION_ASSIGNMENT_V0.read(buffer); + List partitions = new ArrayList<>(); + for (Object structObj : struct.getArray(TOPIC_PARTITIONS_KEY_NAME)) { + Struct assignment = (Struct) structObj; + String topic = assignment.getString(TOPIC_KEY_NAME); + for (Object partitionObj : assignment.getArray(PARTITIONS_KEY_NAME)) { + Integer partition = (Integer) partitionObj; + partitions.add(new TopicPartition(topic, partition)); + } + } + return partitions; + } + private static void checkVersionCompatibility(short version) { // check for invalid versions if (version < CONSUMER_PROTOCOL_V0) From dde1953a310a8b327c047761ed7bd4ee10667f22 Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Sat, 13 May 2017 23:41:20 -0700 Subject: [PATCH 6/9] Fixed minor bugs and improved the stickiness algorithm --- .../kafka/clients/consumer/ConsumerPair.java | 113 ++++++ .../clients/consumer/PartitionMovements.java | 165 ++++++++ .../clients/consumer/StickyAssignor.java | 94 ++++- .../internals/AbstractPartitionAssignor.java | 2 +- .../consumer/internals/PartitionAssignor.java | 2 +- .../clients/consumer/StickyAssignorTest.java | 383 +++++++++++------- 6 files changed, 598 insertions(+), 161 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPair.java create mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/PartitionMovements.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPair.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPair.java new file mode 100644 index 000000000000..f5a476183ae0 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPair.java @@ -0,0 +1,113 @@ +/* + * 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.clients.consumer; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public class ConsumerPair { + protected final String src; + protected final String dst; + + public ConsumerPair(String src, String dst) { + this.src = src; + this.dst = dst; + } + + public String toString() { + return "" + this.src + "->" + this.dst; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((this.src == null) ? 0 : this.src.hashCode()); + result = prime * result + ((this.dst == null) ? 0 : this.dst.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (obj == null) + return false; + + if (!getClass().isInstance(obj)) + return false; + + ConsumerPair otherPair = (ConsumerPair) obj; + return this.src.equals(otherPair.src) && this.dst.equals(otherPair.dst); + } + + public boolean in(Set pairs) { + for (ConsumerPair pair: pairs) + if (this.equals(pair)) + return true; + return false; + } + + private static boolean isLinked(String src, String dst, Set pairs) { + if (src.equals(dst)) + return false; + + if (pairs.isEmpty()) + return false; + + if (new ConsumerPair(src, dst).in(pairs)) + return true; + + for (ConsumerPair pair: pairs) + if (pair.src.equals(src)) { + Set reducedSet = new HashSet<>(pairs); + reducedSet.remove(pair); + if (isLinked(pair.dst, dst, reducedSet)) + return true; + } + + return false; + } + + public static boolean hasCycles(Set pairs) { + for (ConsumerPair pair: pairs) { + Set reducedPairs = new HashSet<>(pairs); + reducedPairs.remove(pair); + if (isLinked(pair.dst, pair.src, reducedPairs)) + return true; + } + return false; + } + + public static boolean hasReversePairs(Set pairs) { + int len = pairs.size(); + if (len < 2) + return false; + + List pairsList = new ArrayList<>(pairs); + for (int i = 0; i < len - 1; ++i) { + ConsumerPair pair1 = pairsList.get(i); + for (int j = i + 1; j < len; ++j) { + ConsumerPair pair2 = pairsList.get(j); + if (pair1.src.equals(pair2.dst) && pair2.src.equals(pair1.dst)) + return true; + } + } + + return false; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/PartitionMovements.java b/clients/src/main/java/org/apache/kafka/clients/consumer/PartitionMovements.java new file mode 100644 index 000000000000..351642d02dbf --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/PartitionMovements.java @@ -0,0 +1,165 @@ +/* + * 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.clients.consumer; + +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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Utils; + +public class PartitionMovements { + private static final Logger log = LoggerFactory.getLogger(PartitionMovements.class); + + private Map>> partitionMovementsByTopic = new HashMap<>(); + private Map partitionMovements = new HashMap<>(); + + private ConsumerPair removeMovementRecordOfPartition(TopicPartition partition) { + ConsumerPair pair = partitionMovements.remove(partition); + + String topic = partition.topic(); + Map> partitionMovementsForThisTopic = partitionMovementsByTopic.get(topic); + partitionMovementsForThisTopic.get(pair).remove(partition); + if (partitionMovementsForThisTopic.get(pair).isEmpty()) + partitionMovementsForThisTopic.remove(pair); + if (partitionMovementsByTopic.get(topic).isEmpty()) + partitionMovementsByTopic.remove(topic); + + return pair; + } + + private void addPartitionMovementRecord(TopicPartition partition, ConsumerPair pair) { + partitionMovements.put(partition, pair); + + String topic = partition.topic(); + if (!partitionMovementsByTopic.containsKey(topic)) + partitionMovementsByTopic.put(topic, new HashMap>()); + + Map> partitionMovementsForThisTopic = partitionMovementsByTopic.get(topic); + if (!partitionMovementsForThisTopic.containsKey(pair)) + partitionMovementsForThisTopic.put(pair, new HashSet()); + + partitionMovementsForThisTopic.get(pair).add(partition); + } + + public void movePartition(TopicPartition partition, String oldConsumer, String newConsumer) { + ConsumerPair pair = new ConsumerPair(oldConsumer, newConsumer); + + if (partitionMovements.containsKey(partition)) { + // this partition has previously moved + ConsumerPair existingPair = removeMovementRecordOfPartition(partition); + assert existingPair.dst.equals(oldConsumer); + if (!existingPair.src.equals(newConsumer)) { + // the partition is not moving back to its previous consumer + // return new ConsumerPair2(existingPair.src, newConsumer); + addPartitionMovementRecord(partition, new ConsumerPair(existingPair.src, newConsumer)); + } + } else + addPartitionMovementRecord(partition, pair); + } + + public TopicPartition getTheActualPartitionToBeMoved(TopicPartition partition, String oldConsumer, String newConsumer) { + String topic = partition.topic(); + + if (!partitionMovementsByTopic.containsKey(topic)) + return partition; + + if (partitionMovements.containsKey(partition)) { + // this partition has previously moved + assert oldConsumer.equals(partitionMovements.get(partition).dst); + oldConsumer = partitionMovements.get(partition).src; + } + + Map> partitionMovementsForThisTopic = partitionMovementsByTopic.get(topic); + ConsumerPair reversePair = new ConsumerPair(newConsumer, oldConsumer); + if (!partitionMovementsForThisTopic.containsKey(reversePair)) + return partition; + + return partitionMovementsForThisTopic.get(reversePair).iterator().next(); + } + + public static boolean verifyStickiness(Map> prevAssignments, + Map> postAssignments) { + if (prevAssignments.isEmpty()) + return true; + + int size = prevAssignments.size(); + + Map> movements = new HashMap<>(); + Map> perTopicMovements = new HashMap<>(); + + List consumers = Utils.sorted(prevAssignments.keySet()); + + for (int i = 0; i < size; ++i) { + String consumer = consumers.get(i); + List prev = prevAssignments.get(consumer); + for (TopicPartition partition: prev) { + + if (postAssignments.containsKey(consumer) && postAssignments.get(consumer).contains(partition)) + // the partition hasn't moved + continue; + + for (int j = 0; j < size; ++j) { + if (j == i) + continue; + + String otherConsumer = consumers.get(j); + if (otherConsumer == null || !postAssignments.containsKey(otherConsumer)) + continue; + + ConsumerPair pair = new ConsumerPair(consumer, otherConsumer); + List post = postAssignments.get(otherConsumer); + if (post.contains(partition)) { + String topic = partition.topic(); + // movement of partition from consumer to otherConsumer + if (movements.get(pair) == null) { + Set moved = new HashSet<>(Collections.singleton(topic)); + movements.put(pair, moved); + } else + movements.get(pair).add(topic); + + // movement per topic + if (perTopicMovements.containsKey(topic)) { + if (!pair.in(perTopicMovements.get(topic))) + perTopicMovements.get(topic).add(pair); + } else + perTopicMovements.put(topic, new HashSet<>(Collections.singleton(pair))); + } else + movements.put(pair, null); + } + } + } + + for (Map.Entry> topicMovements: perTopicMovements.entrySet()) { + if (ConsumerPair.hasReversePairs(topicMovements.getValue())) { + log.error("Stickiness is violated for topic " + topicMovements.getKey() + + "\nPartition movements for this topic occurred between the following consumer pairs:" + + "\n" + topicMovements.getValue().toString()); + return false; + } + } + + return true; + } +} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java index 687b813d0155..6bdbb6a8f7d5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java @@ -167,10 +167,10 @@ * */ public class StickyAssignor extends AbstractPartitionAssignor { - private static final Logger log = LoggerFactory.getLogger(StickyAssignor.class); protected Map> currentAssignment = new HashMap<>(); private List memberAssignment = null; + private PartitionMovements partitionMovements; private void deepCopy(Map> source, Map> dest) { dest.clear(); @@ -178,7 +178,7 @@ private void deepCopy(Map> source, Map(entry.getValue())); } - private Map> deepCopy(Map> assignment) { + protected Map> deepCopy(Map> assignment) { Map> copy = new HashMap<>(); deepCopy(assignment, copy); return copy; @@ -186,6 +186,8 @@ private Map> deepCopy(Map> assign(Map partitionsPerTopic, Map> subscriptions) { + partitionMovements = new PartitionMovements(); + prepopulateCurrentAssignments(); // make a deep copy of currentAssignment Map> oldAssignment = deepCopy(currentAssignment); @@ -242,8 +244,13 @@ public Map> assign(Map partitionsP // if this topic partition of this consumer no longer exists remove it from currentAssignment of the consumer partitionIter.remove(); currentPartitionConsumer.remove(partition); + } else if (!subscriptions.get(entry.getKey()).contains(partition.topic())) { + // if this partition cannot remain assigned to its current consumer because the consumer + // is no longer subscribed to its topic remove it from currentAssignment of the consumer + partitionIter.remove(); } else - // otherwise, remove the topic partition from those that need to be assigned (because it is already assigned + // otherwise, remove the topic partition from those that need to be assigned only if + // its current consumer is still subscribed to its topic (because it is already assigned // and we would want to preserve that assignment as much as possible) unassignedPartitions.remove(partition); } @@ -475,16 +482,6 @@ private String assignPartition(TopicPartition partition, TreeSet sortedC return null; } - /** - * Remove partition from partition assignments of the given consumer - */ - private void unassignPartition(TopicPartition partition, String consumer, TreeSet sortedCurrentSubscriptions, HashMap currentPartitionConsumer) { - sortedCurrentSubscriptions.remove(consumer); - currentAssignment.get(consumer).remove(partition); - currentPartitionConsumer.remove(partition); - sortedCurrentSubscriptions.add(consumer); - } - private boolean canParticipateInReassignment(TopicPartition partition, HashMap> partition2AllPotentialConsumers) { // if a partition has two or more potential consumers it is subject to reassignment. return partition2AllPotentialConsumers.get(partition).size() >= 2; @@ -577,10 +574,10 @@ private boolean performReassignments(List sortedPartitions, Tree HashMap> partition2AllPotentialConsumers, HashMap currentPartitionConsumer) { boolean reassignmentPerformed = false; - boolean modified = true; + boolean modified; // repeat reassignment until no partition can be moved to improve the balance - while (modified) { + do { modified = false; // reassign all reassignable partitions (starting from the partition with least potential consumers and if needed) // until the full list is processed or a balance is achieved @@ -598,22 +595,60 @@ private boolean performReassignments(List sortedPartitions, Tree log.error("Expected partition '" + partition + "' to be assigned to a consumer"); // check if a better-suited consumer exist for the partition; if so, reassign it - for (String otherConsumer: partition2AllPotentialConsumers.get(partition)) + for (String otherConsumer: partition2AllPotentialConsumers.get(partition)) { if (currentAssignment.get(consumer).size() > currentAssignment.get(otherConsumer).size() + 1) { - // de-assign partition from its current consumer - unassignPartition(partition, consumer, sortedCurrentSubscriptions, currentPartitionConsumer); - // reassign the partition to an eligible consumer with fewest assignments - assignPartition(partition, sortedCurrentSubscriptions, consumer2AllPotentialPartitions, currentPartitionConsumer); + reassignPartition(partition, sortedCurrentSubscriptions, currentPartitionConsumer, consumer2AllPotentialPartitions); reassignmentPerformed = true; modified = true; break; } + } } - } + } while (modified); return reassignmentPerformed; } + private void reassignPartition(TopicPartition partition, TreeSet sortedCurrentSubscriptions, + HashMap currentPartitionConsumer, + HashMap> consumer2AllPotentialPartitions) { + String consumer = currentPartitionConsumer.get(partition); + + // find the new consumer + String newConsumer = null; + for (String anotherConsumer: sortedCurrentSubscriptions) { + if (consumer2AllPotentialPartitions.get(anotherConsumer).contains(partition)) { + newConsumer = anotherConsumer; + break; + } + } + + assert newConsumer != null; + + // find the correct partition movement considering the stickiness requirement + TopicPartition partitionToBeMoved = partitionMovements.getTheActualPartitionToBeMoved(partition, consumer, newConsumer); + processPartitionMovement(partitionToBeMoved, newConsumer, sortedCurrentSubscriptions, currentPartitionConsumer); + + return; + } + + private void processPartitionMovement(TopicPartition partition, String newConsumer, + TreeSet sortedCurrentSubscriptions, + HashMap currentPartitionConsumer) { + String oldConsumer = currentPartitionConsumer.get(partition); + + sortedCurrentSubscriptions.remove(oldConsumer); + sortedCurrentSubscriptions.remove(newConsumer); + + partitionMovements.movePartition(partition, oldConsumer, newConsumer); + + currentAssignment.get(oldConsumer).remove(partition); + currentAssignment.get(newConsumer).add(partition); + currentPartitionConsumer.put(partition, newConsumer); + sortedCurrentSubscriptions.add(newConsumer); + sortedCurrentSubscriptions.add(oldConsumer); + } + @SuppressWarnings("serial") private static class PartitionComparator implements Comparator, Serializable { private Map> map; @@ -651,4 +686,21 @@ public int compare(String o1, String o2) { } } + @SuppressWarnings("serial") + protected static class PartitionAssignmentComparator implements Comparator, Serializable { + private Map> map; + + PartitionAssignmentComparator(Map> map) { + this.map = map; + } + + @Override + public int compare(String o1, String o2) { + int ret = map.get(o1).size() - map.get(o2).size(); + if (ret == 0) + ret = o1.compareTo(o2); + return ret; + } + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java index 2fcd3ff235b2..bc87ed025dd2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractPartitionAssignor.java @@ -73,7 +73,7 @@ public Map assign(Cluster metadata, Map> rawAssignments = assign(partitionsPerTopic, topicSubscriptions); - // wrap the results along with any available user data + // this class maintains no user data, so just wrap the results Map assignments = new HashMap<>(); for (Map.Entry> assignmentEntry : rawAssignments.entrySet()) assignments.put(assignmentEntry.getKey(), new Assignment(assignmentEntry.getValue())); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java index 86683a0fa763..4a7c7a8bbd34 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/PartitionAssignor.java @@ -67,7 +67,7 @@ public interface PartitionAssignor { /** - * Unique name for this assignor (e.g. "range" or "roundrobin") + * Unique name for this assignor (e.g. "range" or "roundrobin" or "sticky") * @return non-null unique name */ String name(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java index 4c81d20a23da..42dc3319cd84 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java @@ -17,11 +17,11 @@ package org.apache.kafka.clients.consumer; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -30,9 +30,9 @@ import java.util.Map.Entry; import java.util.Random; import java.util.Set; -import java.util.TreeSet; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Utils; import org.junit.Test; public class StickyAssignorTest { @@ -50,8 +50,8 @@ public void testOneConsumerNoTopic() { assertEquals(Collections.singleton(consumerId), assignment.keySet()); assertTrue(assignment.get(consumerId).isEmpty()); + verifyValidityAndBalance(subscriptions, assignment); assertTrue(isFullyBalanced(assignment)); - verifyBalance(assignment, subscriptions, partitionsPerTopic); } @Test @@ -68,8 +68,8 @@ public void testOneConsumerNonexistentTopic() { assertEquals(Collections.singleton(consumerId), assignment.keySet()); assertTrue(assignment.get(consumerId).isEmpty()); + verifyValidityAndBalance(subscriptions, assignment); assertTrue(isFullyBalanced(assignment)); - verifyBalance(assignment, subscriptions, partitionsPerTopic); } @Test @@ -82,13 +82,10 @@ public void testOneConsumerOneTopic() { Map> subscriptions = Collections.singletonMap(consumerId, topics(topic)); Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - assertEquals(Arrays.asList( - new TopicPartition(topic, 0), - new TopicPartition(topic, 1), - new TopicPartition(topic, 2)), assignment.get(consumerId)); + assertEquals(Arrays.asList(tp(topic, 0), tp(topic, 1), tp(topic, 2)), assignment.get(consumerId)); + verifyValidityAndBalance(subscriptions, assignment); assertTrue(isFullyBalanced(assignment)); - verifyBalance(assignment, subscriptions, partitionsPerTopic); } @Test @@ -103,13 +100,10 @@ public void testOnlyAssignsPartitionsFromSubscribedTopics() { Map> subscriptions = Collections.singletonMap(consumerId, topics(topic)); Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - assertEquals(Arrays.asList( - new TopicPartition(topic, 0), - new TopicPartition(topic, 1), - new TopicPartition(topic, 2)), assignment.get(consumerId)); + assertEquals(Arrays.asList(tp(topic, 0), tp(topic, 1), tp(topic, 2)), assignment.get(consumerId)); + verifyValidityAndBalance(subscriptions, assignment); assertTrue(isFullyBalanced(assignment)); - verifyBalance(assignment, subscriptions, partitionsPerTopic); } @Test @@ -124,13 +118,10 @@ public void testOneConsumerMultipleTopics() { Map> subscriptions = Collections.singletonMap(consumerId, topics(topic1, topic2)); Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - assertEquals(Arrays.asList( - new TopicPartition(topic1, 0), - new TopicPartition(topic2, 0), - new TopicPartition(topic2, 1)), assignment.get(consumerId)); + assertEquals(Arrays.asList(tp(topic1, 0), tp(topic2, 0), tp(topic2, 1)), assignment.get(consumerId)); + verifyValidityAndBalance(subscriptions, assignment); assertTrue(isFullyBalanced(assignment)); - verifyBalance(assignment, subscriptions, partitionsPerTopic); } @Test @@ -147,11 +138,11 @@ public void testTwoConsumersOneTopicOnePartition() { subscriptions.put(consumer2, topics(topic)); Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - assertEquals(Arrays.asList(new TopicPartition(topic, 0)), assignment.get(consumer1)); + assertEquals(Arrays.asList(tp(topic, 0)), assignment.get(consumer1)); assertEquals(Collections.emptyList(), assignment.get(consumer2)); + verifyValidityAndBalance(subscriptions, assignment); assertTrue(isFullyBalanced(assignment)); - verifyBalance(assignment, subscriptions, partitionsPerTopic); } @Test @@ -168,11 +159,11 @@ public void testTwoConsumersOneTopicTwoPartitions() { subscriptions.put(consumer2, topics(topic)); Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - assertEquals(Arrays.asList(new TopicPartition(topic, 0)), assignment.get(consumer1)); - assertEquals(Arrays.asList(new TopicPartition(topic, 1)), assignment.get(consumer2)); + assertEquals(Arrays.asList(tp(topic, 0)), assignment.get(consumer1)); + assertEquals(Arrays.asList(tp(topic, 1)), assignment.get(consumer2)); + verifyValidityAndBalance(subscriptions, assignment); assertTrue(isFullyBalanced(assignment)); - verifyBalance(assignment, subscriptions, partitionsPerTopic); } @Test @@ -193,17 +184,12 @@ public void testMultipleConsumersMixedTopicSubscriptions() { subscriptions.put(consumer3, topics(topic1)); Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - assertEquals(Arrays.asList( - new TopicPartition(topic1, 0), - new TopicPartition(topic1, 2)), assignment.get(consumer1)); - assertEquals(Arrays.asList( - new TopicPartition(topic2, 0), - new TopicPartition(topic2, 1)), assignment.get(consumer2)); - assertEquals(Arrays.asList( - new TopicPartition(topic1, 1)), assignment.get(consumer3)); + assertEquals(Arrays.asList(tp(topic1, 0), tp(topic1, 2)), assignment.get(consumer1)); + assertEquals(Arrays.asList(tp(topic2, 0), tp(topic2, 1)), assignment.get(consumer2)); + assertEquals(Arrays.asList(tp(topic1, 1)), assignment.get(consumer3)); + verifyValidityAndBalance(subscriptions, assignment); assertTrue(isFullyBalanced(assignment)); - verifyBalance(assignment, subscriptions, partitionsPerTopic); } @Test @@ -222,17 +208,11 @@ public void testTwoConsumersTwoTopicsSixPartitions() { subscriptions.put(consumer2, topics(topic1, topic2)); Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - assertEquals(Arrays.asList( - new TopicPartition(topic1, 0), - new TopicPartition(topic1, 2), - new TopicPartition(topic2, 1)), assignment.get(consumer1)); - assertEquals(Arrays.asList( - new TopicPartition(topic1, 1), - new TopicPartition(topic2, 0), - new TopicPartition(topic2, 2)), assignment.get(consumer2)); + assertEquals(Arrays.asList(tp(topic1, 0), tp(topic1, 2), tp(topic2, 1)), assignment.get(consumer1)); + assertEquals(Arrays.asList(tp(topic1, 1), tp(topic2, 0), tp(topic2, 2)), assignment.get(consumer2)); + verifyValidityAndBalance(subscriptions, assignment); assertTrue(isFullyBalanced(assignment)); - verifyBalance(assignment, subscriptions, partitionsPerTopic); } @Test @@ -246,28 +226,32 @@ public void testAddRemoveConsumerOneTopic() { subscriptions.put(consumer1, topics(topic)); Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - assertEquals(Arrays.asList( - new TopicPartition(topic, 0), - new TopicPartition(topic, 1), - new TopicPartition(topic, 2)), assignment.get(consumer1)); + assertEquals(Arrays.asList(tp(topic, 0), tp(topic, 1), tp(topic, 2)), assignment.get(consumer1)); + + verifyValidityAndBalance(subscriptions, assignment); assertTrue(isFullyBalanced(assignment)); - verifyBalance(assignment, subscriptions, partitionsPerTopic); String consumer2 = "consumer2"; subscriptions.put(consumer2, topics(topic)); - Map> assignment2 = assignor.assign(partitionsPerTopic, subscriptions); - assertEquals(Arrays.asList(new TopicPartition(topic, 1), new TopicPartition(topic, 2)), assignment2.get(consumer1)); - assertEquals(Arrays.asList(new TopicPartition(topic, 0)), assignment2.get(consumer2)); - assertTrue(isFullyBalanced(assignment2)); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + Map> oldAssignment = assignor.deepCopy(assignment); + assignment = assignor.assign(partitionsPerTopic, subscriptions); + assertEquals(Arrays.asList(tp(topic, 1), tp(topic, 2)), assignment.get(consumer1)); + assertEquals(Arrays.asList(tp(topic, 0)), assignment.get(consumer2)); + + verifyValidityAndBalance(subscriptions, assignment); + assertTrue(isFullyBalanced(assignment)); + assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); subscriptions.remove(consumer1); - Map> assignment3 = assignor.assign(partitionsPerTopic, subscriptions); - assertTrue(assignment3.get(consumer2).contains(new TopicPartition(topic, 0))); - assertTrue(assignment3.get(consumer2).contains(new TopicPartition(topic, 1))); - assertTrue(assignment3.get(consumer2).contains(new TopicPartition(topic, 2))); - assertTrue(isFullyBalanced(assignment3)); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + oldAssignment = assignor.deepCopy(assignment); + assignment = assignor.assign(partitionsPerTopic, subscriptions); + assertTrue(assignment.get(consumer2).contains(tp(topic, 0))); + assertTrue(assignment.get(consumer2).contains(tp(topic, 1))); + assertTrue(assignment.get(consumer2).contains(tp(topic, 2))); + + verifyValidityAndBalance(subscriptions, assignment); + assertTrue(isFullyBalanced(assignment)); + assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); } /** @@ -302,7 +286,7 @@ public void testPoorRoundRobinAssignmentScenario() { subscriptions.put("consumer4", Arrays.asList("topic1", "topic2", "topic3", "topic4", "topic5")); Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); } @Test @@ -320,7 +304,7 @@ public void testAddRemoveTopicTwoConsumers() { Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); // verify balance assertTrue(isFullyBalanced(assignment)); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); // verify stickiness List consumer1Assignment1 = assignment.get(consumer1); List consumer2Assignment1 = assignment.get(consumer2); @@ -331,31 +315,35 @@ public void testAddRemoveTopicTwoConsumers() { partitionsPerTopic.put(topic2, 3); subscriptions.put(consumer1, topics(topic, topic2)); subscriptions.put(consumer2, topics(topic, topic2)); - Map> assignment2 = assignor.assign(partitionsPerTopic, subscriptions); + Map> oldAssignment = assignor.deepCopy(assignment); + assignment = assignor.assign(partitionsPerTopic, subscriptions); // verify balance - assertTrue(isFullyBalanced(assignment2)); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); + assertTrue(isFullyBalanced(assignment)); // verify stickiness - List consumer1Assignment2 = assignment2.get(consumer1); - List consumer2Assignment2 = assignment2.get(consumer2); - assertTrue(consumer1Assignment2.size() == 3 && consumer2Assignment2.size() == 3); - assertTrue(consumer1Assignment2.containsAll(consumer1Assignment1)); - assertTrue(consumer2Assignment2.containsAll(consumer2Assignment1)); + List consumer1assignment = assignment.get(consumer1); + List consumer2assignment = assignment.get(consumer2); + assertTrue(consumer1assignment.size() == 3 && consumer2assignment.size() == 3); + assertTrue(consumer1assignment.containsAll(consumer1Assignment1)); + assertTrue(consumer2assignment.containsAll(consumer2Assignment1)); + assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); partitionsPerTopic.remove(topic); subscriptions.put(consumer1, topics(topic2)); subscriptions.put(consumer2, topics(topic2)); - Map> assignment3 = assignor.assign(partitionsPerTopic, subscriptions); + oldAssignment = assignor.deepCopy(assignment); + assignment = assignor.assign(partitionsPerTopic, subscriptions); // verify balance - assertTrue(isFullyBalanced(assignment3)); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); + assertTrue(isFullyBalanced(assignment)); // verify stickiness - List consumer1Assignment3 = assignment3.get(consumer1); - List consumer2Assignment3 = assignment3.get(consumer2); + List consumer1Assignment3 = assignment.get(consumer1); + List consumer2Assignment3 = assignment.get(consumer2); assertTrue((consumer1Assignment3.size() == 1 && consumer2Assignment3.size() == 2) || (consumer1Assignment3.size() == 2 && consumer2Assignment3.size() == 1)); - assertTrue(consumer1Assignment2.containsAll(consumer1Assignment3)); - assertTrue(consumer2Assignment2.containsAll(consumer2Assignment3)); + assertTrue(consumer1assignment.containsAll(consumer1Assignment3)); + assertTrue(consumer2assignment.containsAll(consumer2Assignment3)); + assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); } @Test @@ -373,11 +361,13 @@ public void testReassignmentAfterOneConsumerLeaves() { } Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); subscriptions.remove("consumer10"); + Map> oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); + assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); } @Test @@ -390,11 +380,13 @@ public void testReassignmentAfterOneConsumerAdded() { subscriptions.put(String.format("consumer%02d", i), Collections.singletonList("topic")); Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); subscriptions.put("consumer10", Collections.singletonList("topic")); + Map> oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); + assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); } @Test @@ -412,41 +404,45 @@ public void testSameSubscriptions() { } Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); subscriptions.remove("consumer05"); + Map> oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); + assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); } @Test public void testLargeAssignmentWithMultipleConsumersLeaving() { Random rand = new Random(); - int topicCount = 400; - int consumerCount = 1000; + int topicCount = 40; + int consumerCount = 200; Map partitionsPerTopic = new HashMap<>(); for (int i = 0; i < topicCount; i++) - partitionsPerTopic.put(String.format("t%d", i), rand.nextInt(10) + 1); + partitionsPerTopic.put(getTopicName(i, topicCount), rand.nextInt(10) + 1); Map> subscriptions = new HashMap<>(); for (int i = 0; i < consumerCount; i++) { List topics = new ArrayList(); for (int j = 0; j < rand.nextInt(20); j++) - topics.add(String.format("t%d", rand.nextInt(topicCount))); - subscriptions.put(String.format("c%d", i), topics); + topics.add(getTopicName(rand.nextInt(topicCount), topicCount)); + subscriptions.put(getConsumerName(i, consumerCount), topics); } Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); for (int i = 0; i < 100; ++i) { - String c = String.format("c%d", rand.nextInt(consumerCount)); + String c = getConsumerName(rand.nextInt(consumerCount), consumerCount); subscriptions.remove(c); } + Map> oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); + assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); } @Test @@ -464,12 +460,81 @@ public void testNewSubscription() { } Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); subscriptions.get("consumer00").add("topic01"); + Map> oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); + assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); + } + + @Test + public void testReassignmentWithRandomSubscriptionsAndChanges() { + final int minNumConsumers = 20; + final int maxNumConsumers = 40; + final int minNumTopics = 10; + final int maxNumTopics = 20; + + for (int round = 1; round <= 100; ++round) { + int numTopics = minNumTopics + new Random().nextInt(maxNumTopics - minNumTopics); + + ArrayList topics = new ArrayList<>(); + for (int i = 0; i < numTopics; ++i) + topics.add(getTopicName(i, maxNumTopics)); + + Map partitionsPerTopic = new HashMap<>(); + for (int i = 0; i < numTopics; ++i) + partitionsPerTopic.put(getTopicName(i, maxNumTopics), i + 1); + + int numConsumers = minNumConsumers + new Random().nextInt(maxNumConsumers - minNumConsumers); + + Map> subscriptions = new HashMap<>(); + for (int i = 0; i < numConsumers; ++i) { + List sub = Utils.sorted(getRandomSublist(topics)); + subscriptions.put(getConsumerName(i, maxNumConsumers), sub); + } + + StickyAssignor assignor = new StickyAssignor(); + + Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyValidityAndBalance(subscriptions, assignment); + + subscriptions.clear(); + for (int i = 0; i < numConsumers; ++i) { + List sub = Utils.sorted(getRandomSublist(topics)); + subscriptions.put(getConsumerName(i, maxNumConsumers), sub); + } + + Map> oldAssignment = assignor.deepCopy(assignment); + assignment = assignor.assign(partitionsPerTopic, subscriptions); + verifyValidityAndBalance(subscriptions, assignment); + assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); + } + } + + private String getTopicName(int i, int maxNum) { + return getCanonicalName("t", i, maxNum); + } + + private String getConsumerName(int i, int maxNum) { + return getCanonicalName("c", i, maxNum); + } + + private String getCanonicalName(String str, int i, int maxNum) { + return str + pad(i, Integer.toString(maxNum).length()); + } + + private String pad(int num, int digits) { + StringBuilder sb = new StringBuilder(); + int iDigits = Integer.toString(num).length(); + + for (int i = 1; i <= digits - iDigits; ++i) + sb.append("0"); + + sb.append(num); + return sb.toString(); } @Test @@ -483,12 +548,12 @@ public void testMoveExistingAssignments() { subscriptions.put("consumer02", topics("topic01", "topic02", "topic03", "topic04")); subscriptions.put("consumer03", topics("topic02", "topic03", "topic04", "topic05", "topic06")); - assignor.currentAssignment.put("consumer01", new ArrayList<>(Arrays.asList(new TopicPartition("topic01", 0)))); - assignor.currentAssignment.put("consumer02", new ArrayList<>(Arrays.asList(new TopicPartition("topic02", 0), new TopicPartition("topic03", 0)))); - assignor.currentAssignment.put("consumer03", new ArrayList<>(Arrays.asList(new TopicPartition("topic04", 0), new TopicPartition("topic05", 0), new TopicPartition("topic06", 0)))); + assignor.currentAssignment.put("consumer01", new ArrayList<>(Arrays.asList(tp("topic01", 0)))); + assignor.currentAssignment.put("consumer02", new ArrayList<>(Arrays.asList(tp("topic02", 0), tp("topic03", 0)))); + assignor.currentAssignment.put("consumer03", new ArrayList<>(Arrays.asList(tp("topic04", 0), tp("topic05", 0), tp("topic06", 0)))); Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); } @Test @@ -502,7 +567,7 @@ public void testStickiness() { subscriptions.put("consumer04", topics("topic01")); Map> assignment = assignor.assign(partitionsPerTopic, subscriptions); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); Map partitionsAssigned = new HashMap<>(); Set>> assignments = assignment.entrySet(); @@ -518,8 +583,10 @@ public void testStickiness() { // removing the potential group leader subscriptions.remove("consumer01"); + Map> oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); - verifyBalance(assignment, subscriptions, partitionsPerTopic); + verifyValidityAndBalance(subscriptions, assignment); + assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); assignments = assignment.entrySet(); for (Map.Entry> entry: assignments) { @@ -539,7 +606,7 @@ public static TopicPartition tp(String topic, int partition) { return new TopicPartition(topic, partition); } - private boolean isFullyBalanced(Map> assignment) { + private static boolean isFullyBalanced(Map> assignment) { int min = Integer.MAX_VALUE; int max = Integer.MIN_VALUE; for (List topicPartitions: assignment.values()) { @@ -552,56 +619,96 @@ private boolean isFullyBalanced(Map> assignment) { return max - min <= 1; } + private static List getRandomSublist(ArrayList list) { + List selectedItems = new ArrayList<>(list); + int len = list.size(); + Random random = new Random(); + int howManyToRemove = random.nextInt(len); + + for (int i = 1; i <= howManyToRemove; ++i) + selectedItems.remove(random.nextInt(selectedItems.size())); + + return selectedItems; + } + /** - * Verify that either: - * - the given assignment is fully balanced (the numbers of topic partitions assigned to consumers differ by at most one), or - * - there is no topic partition that can be moved from one consumer to another that has 2+ fewer topic partitions + * Verifies that the given assignment is valid and balanced with respect to the given subscriptions + * Validity requirements: + * - each consumer is subscribed to topics of all partitions assigned to it, and + * - each partition is assigned to no more than one consumer + * Balance requirements: + * - the assignment is fully balanced (the numbers of topic partitions assigned to consumers differ by at most one), or + * - there is no topic partition that can be moved from one consumer to another with 2+ fewer topic partitions * - * @param assignment: given assignment for balance check * @param subscriptions: topic subscriptions of each consumer - * @param partitionsPerTopic: number of partitions per topic + * @param assignment: given assignment for balance check */ - private void verifyBalance(Map> assignment, Map> subscriptions, Map partitionsPerTopic) { - if (isFullyBalanced(assignment)) - return; - - // an ascending sorted set of consumer based on how many topic partitions are assigned to them in the given assignment - TreeSet consumers = new TreeSet(new StickyAssignor.SubscriptionComparator(assignment)); - consumers.addAll(assignment.keySet()); - - // all possible assignments - final HashMap> allSubscriptions = new HashMap<>(); - for (Map.Entry> entry: subscriptions.entrySet()) { - String consumer = entry.getKey(); - Set topicPartitions = new HashSet<>(); - allSubscriptions.put(consumer, topicPartitions); - for (String topic: entry.getValue()) { - for (int i = 0; i < partitionsPerTopic.get(topic); i++) { - TopicPartition topicPartition = new TopicPartition(topic, i); - topicPartitions.add(topicPartition); + private static void verifyValidityAndBalance(Map> subscriptions, Map> assignments) { + int size = subscriptions.size(); + assert size == assignments.size(); + + List consumers = Utils.sorted(assignments.keySet()); + + for (int i = 0; i < size; ++i) { + String consumer = consumers.get(i); + List partitions = assignments.get(consumer); + for (TopicPartition partition: partitions) + assertTrue("Error: Partition " + partition + "is assigned to c" + i + ", but it is not subscribed to Topic t" + partition.topic() + + "\nSubscriptions: " + subscriptions.toString() + "\nAssignments: " + assignments.toString(), + subscriptions.get(consumer).contains(partition.topic())); + + if (i == size - 1) + continue; + + for (int j = i + 1; j < size; ++j) { + String otherConsumer = consumers.get(j); + List otherPartitions = assignments.get(otherConsumer); + + Set intersection = new HashSet<>(partitions); + intersection.retainAll(otherPartitions); + assertTrue("Error: Consumers c" + i + " and c" + j + " have common partitions assigned to them: " + intersection.toString() + + "\nSubscriptions: " + subscriptions.toString() + "\nAssignments: " + assignments.toString(), + intersection.isEmpty()); + + int len = partitions.size(); + int otherLen = otherPartitions.size(); + + if (Math.abs(len - otherLen) <= 1) + continue; + + Map> map = asMap(partitions); + Map> otherMap = asMap(otherPartitions); + + if (len > otherLen) { + for (String topic: map.keySet()) + assertTrue("Error: Some partitions can be moved from c" + i + " to c" + j + " to achieve a better balance" + + "\nc" + i + " has " + len + " partitions, and c" + j + " has " + otherLen + " partitions." + + "\nSubscriptions: " + subscriptions.toString() + "\nAssignments: " + assignments.toString(), + !otherMap.containsKey(topic)); } - } - } - // create a mapping from partitions to the consumer assigned to them - final HashMap allPartitions = new HashMap<>(); - for (Map.Entry> entry: assignment.entrySet()) { - for (TopicPartition topicPartition: entry.getValue()) { - assertTrue(topicPartition + " is assigned to more than one consumer.", !allPartitions.containsKey(topicPartition)); - allPartitions.put(topicPartition, entry.getKey()); + if (otherLen > len) { + for (String topic: otherMap.keySet()) + assertTrue("Error: Some partitions can be moved from c" + j + " to c" + i + " to achieve a better balance" + + "\nc" + i + " has " + len + " partitions, and c" + j + " has " + otherLen + " partitions." + + "\nSubscriptions: " + subscriptions.toString() + "\nAssignments: " + assignments.toString(), + !map.containsKey(topic)); + } } } + } - // starting from the consumer with fewest assignments make sure there is no topic partition that could be - // assigned to this consumer and it is not (because it is assigned to a consumer with more topic partitions) - for (String consumer: consumers) { - for (TopicPartition topicPartition: allSubscriptions.get(consumer)) { - if (!assignment.get(consumer).contains(topicPartition)) { - String otherConsumer = allPartitions.get(topicPartition); - assertNotNull(otherConsumer); - assertTrue(topicPartition + " can be assigned to another consumer for a more balanced assignment", assignment.get(consumer).size() >= assignment.get(otherConsumer).size() - 1); - } + private static Map> asMap(Collection partitions) { + Map> partitionMap = new HashMap<>(); + for (TopicPartition partition : partitions) { + String topic = partition.topic(); + List topicPartitions = partitionMap.get(topic); + if (topicPartitions == null) { + topicPartitions = new ArrayList<>(); + partitionMap.put(topic, topicPartitions); } + topicPartitions.add(partition.partition()); } + return partitionMap; } } From 0810e6897210595d138ba171fe00ae1b171cc69e Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Tue, 16 May 2017 16:24:57 -0700 Subject: [PATCH 7/9] Addressed feedback received on 05/16 - Moved `ConsumerPair` and `PartitionMovements` definitions inside `StickyAssignor` - Moved the schema definition for topic partition list and its serializer/deserializer inside `StickyAssignor` - Limited the exposure of newly added class members --- .../kafka/clients/consumer/ConsumerPair.java | 113 -------- .../clients/consumer/PartitionMovements.java | 165 ----------- .../clients/consumer/StickyAssignor.java | 271 +++++++++++++++++- .../consumer/internals/ConsumerProtocol.java | 42 +-- .../apache/kafka/common/TopicPartition.java | 1 - .../clients/consumer/StickyAssignorTest.java | 55 +--- 6 files changed, 275 insertions(+), 372 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPair.java delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/PartitionMovements.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPair.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPair.java deleted file mode 100644 index f5a476183ae0..000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerPair.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * 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.clients.consumer; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; - -public class ConsumerPair { - protected final String src; - protected final String dst; - - public ConsumerPair(String src, String dst) { - this.src = src; - this.dst = dst; - } - - public String toString() { - return "" + this.src + "->" + this.dst; - } - - @Override - public int hashCode() { - final int prime = 31; - int result = 1; - result = prime * result + ((this.src == null) ? 0 : this.src.hashCode()); - result = prime * result + ((this.dst == null) ? 0 : this.dst.hashCode()); - return result; - } - - @Override - public boolean equals(Object obj) { - if (obj == null) - return false; - - if (!getClass().isInstance(obj)) - return false; - - ConsumerPair otherPair = (ConsumerPair) obj; - return this.src.equals(otherPair.src) && this.dst.equals(otherPair.dst); - } - - public boolean in(Set pairs) { - for (ConsumerPair pair: pairs) - if (this.equals(pair)) - return true; - return false; - } - - private static boolean isLinked(String src, String dst, Set pairs) { - if (src.equals(dst)) - return false; - - if (pairs.isEmpty()) - return false; - - if (new ConsumerPair(src, dst).in(pairs)) - return true; - - for (ConsumerPair pair: pairs) - if (pair.src.equals(src)) { - Set reducedSet = new HashSet<>(pairs); - reducedSet.remove(pair); - if (isLinked(pair.dst, dst, reducedSet)) - return true; - } - - return false; - } - - public static boolean hasCycles(Set pairs) { - for (ConsumerPair pair: pairs) { - Set reducedPairs = new HashSet<>(pairs); - reducedPairs.remove(pair); - if (isLinked(pair.dst, pair.src, reducedPairs)) - return true; - } - return false; - } - - public static boolean hasReversePairs(Set pairs) { - int len = pairs.size(); - if (len < 2) - return false; - - List pairsList = new ArrayList<>(pairs); - for (int i = 0; i < len - 1; ++i) { - ConsumerPair pair1 = pairsList.get(i); - for (int j = i + 1; j < len; ++j) { - ConsumerPair pair2 = pairsList.get(j); - if (pair1.src.equals(pair2.dst) && pair2.src.equals(pair1.dst)) - return true; - } - } - - return false; - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/PartitionMovements.java b/clients/src/main/java/org/apache/kafka/clients/consumer/PartitionMovements.java deleted file mode 100644 index 351642d02dbf..000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/PartitionMovements.java +++ /dev/null @@ -1,165 +0,0 @@ -/* - * 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.clients.consumer; - -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 org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.Utils; - -public class PartitionMovements { - private static final Logger log = LoggerFactory.getLogger(PartitionMovements.class); - - private Map>> partitionMovementsByTopic = new HashMap<>(); - private Map partitionMovements = new HashMap<>(); - - private ConsumerPair removeMovementRecordOfPartition(TopicPartition partition) { - ConsumerPair pair = partitionMovements.remove(partition); - - String topic = partition.topic(); - Map> partitionMovementsForThisTopic = partitionMovementsByTopic.get(topic); - partitionMovementsForThisTopic.get(pair).remove(partition); - if (partitionMovementsForThisTopic.get(pair).isEmpty()) - partitionMovementsForThisTopic.remove(pair); - if (partitionMovementsByTopic.get(topic).isEmpty()) - partitionMovementsByTopic.remove(topic); - - return pair; - } - - private void addPartitionMovementRecord(TopicPartition partition, ConsumerPair pair) { - partitionMovements.put(partition, pair); - - String topic = partition.topic(); - if (!partitionMovementsByTopic.containsKey(topic)) - partitionMovementsByTopic.put(topic, new HashMap>()); - - Map> partitionMovementsForThisTopic = partitionMovementsByTopic.get(topic); - if (!partitionMovementsForThisTopic.containsKey(pair)) - partitionMovementsForThisTopic.put(pair, new HashSet()); - - partitionMovementsForThisTopic.get(pair).add(partition); - } - - public void movePartition(TopicPartition partition, String oldConsumer, String newConsumer) { - ConsumerPair pair = new ConsumerPair(oldConsumer, newConsumer); - - if (partitionMovements.containsKey(partition)) { - // this partition has previously moved - ConsumerPair existingPair = removeMovementRecordOfPartition(partition); - assert existingPair.dst.equals(oldConsumer); - if (!existingPair.src.equals(newConsumer)) { - // the partition is not moving back to its previous consumer - // return new ConsumerPair2(existingPair.src, newConsumer); - addPartitionMovementRecord(partition, new ConsumerPair(existingPair.src, newConsumer)); - } - } else - addPartitionMovementRecord(partition, pair); - } - - public TopicPartition getTheActualPartitionToBeMoved(TopicPartition partition, String oldConsumer, String newConsumer) { - String topic = partition.topic(); - - if (!partitionMovementsByTopic.containsKey(topic)) - return partition; - - if (partitionMovements.containsKey(partition)) { - // this partition has previously moved - assert oldConsumer.equals(partitionMovements.get(partition).dst); - oldConsumer = partitionMovements.get(partition).src; - } - - Map> partitionMovementsForThisTopic = partitionMovementsByTopic.get(topic); - ConsumerPair reversePair = new ConsumerPair(newConsumer, oldConsumer); - if (!partitionMovementsForThisTopic.containsKey(reversePair)) - return partition; - - return partitionMovementsForThisTopic.get(reversePair).iterator().next(); - } - - public static boolean verifyStickiness(Map> prevAssignments, - Map> postAssignments) { - if (prevAssignments.isEmpty()) - return true; - - int size = prevAssignments.size(); - - Map> movements = new HashMap<>(); - Map> perTopicMovements = new HashMap<>(); - - List consumers = Utils.sorted(prevAssignments.keySet()); - - for (int i = 0; i < size; ++i) { - String consumer = consumers.get(i); - List prev = prevAssignments.get(consumer); - for (TopicPartition partition: prev) { - - if (postAssignments.containsKey(consumer) && postAssignments.get(consumer).contains(partition)) - // the partition hasn't moved - continue; - - for (int j = 0; j < size; ++j) { - if (j == i) - continue; - - String otherConsumer = consumers.get(j); - if (otherConsumer == null || !postAssignments.containsKey(otherConsumer)) - continue; - - ConsumerPair pair = new ConsumerPair(consumer, otherConsumer); - List post = postAssignments.get(otherConsumer); - if (post.contains(partition)) { - String topic = partition.topic(); - // movement of partition from consumer to otherConsumer - if (movements.get(pair) == null) { - Set moved = new HashSet<>(Collections.singleton(topic)); - movements.put(pair, moved); - } else - movements.get(pair).add(topic); - - // movement per topic - if (perTopicMovements.containsKey(topic)) { - if (!pair.in(perTopicMovements.get(topic))) - perTopicMovements.get(topic).add(pair); - } else - perTopicMovements.put(topic, new HashSet<>(Collections.singleton(pair))); - } else - movements.put(pair, null); - } - } - } - - for (Map.Entry> topicMovements: perTopicMovements.entrySet()) { - if (ConsumerPair.hasReversePairs(topicMovements.getValue())) { - log.error("Stickiness is violated for topic " + topicMovements.getKey() - + "\nPartition movements for this topic occurred between the following consumer pairs:" - + "\n" + topicMovements.getValue().toString()); - return false; - } - } - - return true; - } -} diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java index 6bdbb6a8f7d5..387cf3574030 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java @@ -20,6 +20,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -33,6 +34,11 @@ import org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor; import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.protocol.types.ArrayOf; +import org.apache.kafka.common.protocol.types.Field; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.SchemaException; +import org.apache.kafka.common.protocol.types.Struct; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -168,6 +174,12 @@ */ public class StickyAssignor extends AbstractPartitionAssignor { private static final Logger log = LoggerFactory.getLogger(StickyAssignor.class); + + // this schema is used for preserving consumer's assigned partitions list and + // sending it to the leader during a rebalance + public static final Schema TOPIC_PARTITION_ASSIGNMENT_V0 = new Schema( + new Field(ConsumerProtocol.TOPIC_PARTITIONS_KEY_NAME, new ArrayOf(ConsumerProtocol.TOPIC_ASSIGNMENT_V0))); + protected Map> currentAssignment = new HashMap<>(); private List memberAssignment = null; private PartitionMovements partitionMovements; @@ -178,7 +190,7 @@ private void deepCopy(Map> source, Map(entry.getValue())); } - protected Map> deepCopy(Map> assignment) { + private Map> deepCopy(Map> assignment) { Map> copy = new HashMap<>(); deepCopy(assignment, copy); return copy; @@ -277,7 +289,7 @@ private void prepopulateCurrentAssignments() { for (Map.Entry subscriptionEntry : subscriptions.entrySet()) { ByteBuffer userData = subscriptionEntry.getValue().userData(); if (userData != null && userData.hasRemaining()) - currentAssignment.put(subscriptionEntry.getKey(), ConsumerProtocol.deserializeTopicPartitionAssignment(userData)); + currentAssignment.put(subscriptionEntry.getKey(), deserializeTopicPartitionAssignment(userData)); } } @@ -291,7 +303,7 @@ public Subscription subscription(Set topics) { if (memberAssignment == null) return new Subscription(new ArrayList<>(topics)); - return new Subscription(new ArrayList<>(topics), ConsumerProtocol.serializeTopicPartitionAssignment(memberAssignment)); + return new Subscription(new ArrayList<>(topics), serializeTopicPartitionAssignment(memberAssignment)); } @Override @@ -306,7 +318,7 @@ public String name() { * @param allSubscriptions: a mapping of all consumers to all potential topic partitions that can be assigned to them * @return */ - protected boolean isBalanced(TreeSet sortedCurrentSubscriptions, Map> allSubscriptions) { + private boolean isBalanced(TreeSet sortedCurrentSubscriptions, Map> allSubscriptions) { int min = currentAssignment.get(sortedCurrentSubscriptions.first()).size(); int max = currentAssignment.get(sortedCurrentSubscriptions.last()).size(); if (min >= max - 1) @@ -356,7 +368,7 @@ protected boolean isBalanced(TreeSet sortedCurrentSubscriptions, Map> assignment) { + private int getBalanceScore(Map> assignment) { int score = 0; Map consumer2AssignmentSize = new HashMap<>(); @@ -528,12 +540,10 @@ private void balance(List sortedPartitions, List } // narrow down the reassignment scope to only those partitions that can actually be reassigned - Set reassignablePartitions = new HashSet<>(partition2AllPotentialConsumers.keySet()); Set fixedPartitions = new HashSet<>(); - for (TopicPartition partition: reassignablePartitions) + for (TopicPartition partition: partition2AllPotentialConsumers.keySet()) if (!canParticipateInReassignment(partition, partition2AllPotentialConsumers)) fixedPartitions.add(partition); - reassignablePartitions.removeAll(fixedPartitions); sortedPartitions.removeAll(fixedPartitions); // narrow down the reassignment scope to only those consumers that are subject to reassignment @@ -569,7 +579,7 @@ private void balance(List sortedPartitions, List fixedAssignments.clear(); } - private boolean performReassignments(List sortedPartitions, TreeSet sortedCurrentSubscriptions, + private boolean performReassignments(List reassignablePartitions, TreeSet sortedCurrentSubscriptions, HashMap> consumer2AllPotentialPartitions, HashMap> partition2AllPotentialConsumers, HashMap currentPartitionConsumer) { @@ -581,7 +591,7 @@ private boolean performReassignments(List sortedPartitions, Tree modified = false; // reassign all reassignable partitions (starting from the partition with least potential consumers and if needed) // until the full list is processed or a balance is achieved - Iterator partitionIterator = sortedPartitions.iterator(); + Iterator partitionIterator = reassignablePartitions.iterator(); while (partitionIterator.hasNext() && !isBalanced(sortedCurrentSubscriptions, consumer2AllPotentialPartitions)) { TopicPartition partition = partitionIterator.next(); @@ -649,8 +659,48 @@ private void processPartitionMovement(TopicPartition partition, String newConsum sortedCurrentSubscriptions.add(oldConsumer); } - @SuppressWarnings("serial") + public boolean isSticky() { + return partitionMovements.isSticky(); + } + + private static ByteBuffer serializeTopicPartitionAssignment(List partitions) { + Struct struct = new Struct(TOPIC_PARTITION_ASSIGNMENT_V0); + List topicAssignments = new ArrayList<>(); + for (Map.Entry> topicEntry : ConsumerProtocol.asMap(partitions).entrySet()) { + Struct topicAssignment = new Struct(ConsumerProtocol.TOPIC_ASSIGNMENT_V0); + topicAssignment.set(ConsumerProtocol.TOPIC_KEY_NAME, topicEntry.getKey()); + topicAssignment.set(ConsumerProtocol.PARTITIONS_KEY_NAME, topicEntry.getValue().toArray()); + topicAssignments.add(topicAssignment); + } + struct.set(ConsumerProtocol.TOPIC_PARTITIONS_KEY_NAME, topicAssignments.toArray()); + ByteBuffer buffer = ByteBuffer.allocate(ConsumerProtocol.CONSUMER_PROTOCOL_HEADER_V0.sizeOf() + + TOPIC_PARTITION_ASSIGNMENT_V0.sizeOf(struct)); + ConsumerProtocol.CONSUMER_PROTOCOL_HEADER_V0.writeTo(buffer); + TOPIC_PARTITION_ASSIGNMENT_V0.write(buffer, struct); + buffer.flip(); + return buffer; + } + + private static List deserializeTopicPartitionAssignment(ByteBuffer buffer) { + Struct header = ConsumerProtocol.CONSUMER_PROTOCOL_HEADER_SCHEMA.read(buffer); + Short version = header.getShort(ConsumerProtocol.VERSION_KEY_NAME); + if (version < ConsumerProtocol.CONSUMER_PROTOCOL_V0) + throw new SchemaException("Unsupported subscription version: " + version); + Struct struct = TOPIC_PARTITION_ASSIGNMENT_V0.read(buffer); + List partitions = new ArrayList<>(); + for (Object structObj : struct.getArray(ConsumerProtocol.TOPIC_PARTITIONS_KEY_NAME)) { + Struct assignment = (Struct) structObj; + String topic = assignment.getString(ConsumerProtocol.TOPIC_KEY_NAME); + for (Object partitionObj : assignment.getArray(ConsumerProtocol.PARTITIONS_KEY_NAME)) { + Integer partition = (Integer) partitionObj; + partitions.add(new TopicPartition(topic, partition)); + } + } + return partitions; + } + private static class PartitionComparator implements Comparator, Serializable { + private static final long serialVersionUID = 1L; private Map> map; PartitionComparator(Map> map) { @@ -669,8 +719,8 @@ public int compare(TopicPartition o1, TopicPartition o2) { } } - @SuppressWarnings("serial") protected static class SubscriptionComparator implements Comparator, Serializable { + private static final long serialVersionUID = 1L; private Map> map; SubscriptionComparator(Map> map) { @@ -686,8 +736,8 @@ public int compare(String o1, String o2) { } } - @SuppressWarnings("serial") protected static class PartitionAssignmentComparator implements Comparator, Serializable { + private static final long serialVersionUID = 1L; private Map> map; PartitionAssignmentComparator(Map> map) { @@ -703,4 +753,199 @@ public int compare(String o1, String o2) { } } + /** + * This class maintains some data structures to simplify lookup of partition movements among consumers. At each point of + * time during a partition rebalance it keeps track of partition movements corresponding to each topic, and also possible + * movement (in form a ConsumerPair object) for each partition. + */ + private static class PartitionMovements { + private Map>> partitionMovementsByTopic = new HashMap<>(); + private Map partitionMovements = new HashMap<>(); + + private ConsumerPair removeMovementRecordOfPartition(TopicPartition partition) { + ConsumerPair pair = partitionMovements.remove(partition); + + String topic = partition.topic(); + Map> partitionMovementsForThisTopic = partitionMovementsByTopic.get(topic); + partitionMovementsForThisTopic.get(pair).remove(partition); + if (partitionMovementsForThisTopic.get(pair).isEmpty()) + partitionMovementsForThisTopic.remove(pair); + if (partitionMovementsByTopic.get(topic).isEmpty()) + partitionMovementsByTopic.remove(topic); + + return pair; + } + + private void addPartitionMovementRecord(TopicPartition partition, ConsumerPair pair) { + partitionMovements.put(partition, pair); + + String topic = partition.topic(); + if (!partitionMovementsByTopic.containsKey(topic)) + partitionMovementsByTopic.put(topic, new HashMap>()); + + Map> partitionMovementsForThisTopic = partitionMovementsByTopic.get(topic); + if (!partitionMovementsForThisTopic.containsKey(pair)) + partitionMovementsForThisTopic.put(pair, new HashSet()); + + partitionMovementsForThisTopic.get(pair).add(partition); + } + + private void movePartition(TopicPartition partition, String oldConsumer, String newConsumer) { + ConsumerPair pair = new ConsumerPair(oldConsumer, newConsumer); + + if (partitionMovements.containsKey(partition)) { + // this partition has previously moved + ConsumerPair existingPair = removeMovementRecordOfPartition(partition); + assert existingPair.dst.equals(oldConsumer); + if (!existingPair.src.equals(newConsumer)) { + // the partition is not moving back to its previous consumer + // return new ConsumerPair2(existingPair.src, newConsumer); + addPartitionMovementRecord(partition, new ConsumerPair(existingPair.src, newConsumer)); + } + } else + addPartitionMovementRecord(partition, pair); + } + + private TopicPartition getTheActualPartitionToBeMoved(TopicPartition partition, String oldConsumer, String newConsumer) { + String topic = partition.topic(); + + if (!partitionMovementsByTopic.containsKey(topic)) + return partition; + + if (partitionMovements.containsKey(partition)) { + // this partition has previously moved + assert oldConsumer.equals(partitionMovements.get(partition).dst); + oldConsumer = partitionMovements.get(partition).src; + } + + Map> partitionMovementsForThisTopic = partitionMovementsByTopic.get(topic); + ConsumerPair reversePair = new ConsumerPair(newConsumer, oldConsumer); + if (!partitionMovementsForThisTopic.containsKey(reversePair)) + return partition; + + return partitionMovementsForThisTopic.get(reversePair).iterator().next(); + } + + private boolean isLinked(String src, String dst, Set pairs, List currentPath) { + if (src.equals(dst)) + return false; + + if (pairs.isEmpty()) + return false; + + if (new ConsumerPair(src, dst).in(pairs)) { + currentPath.add(src); + currentPath.add(dst); + return true; + } + + for (ConsumerPair pair: pairs) + if (pair.src.equals(src)) { + Set reducedSet = new HashSet<>(pairs); + reducedSet.remove(pair); + currentPath.add(pair.src); + return isLinked(pair.dst, dst, reducedSet, currentPath); + } + + return false; + } + + private boolean in(List cycle, Set> cycles) { + List superCycle = new ArrayList<>(cycle); + superCycle.remove(superCycle.size() - 1); + superCycle.addAll(cycle); + Iterator> it = cycles.iterator(); + while (it.hasNext()) { + List next = it.next(); + if (next.size() == cycle.size() && Collections.indexOfSubList(superCycle, next) != -1) + return true; + } + return false; + } + + private boolean hasCycles(Set pairs) { + Set> cycles = new HashSet<>(); + for (ConsumerPair pair: pairs) { + Set reducedPairs = new HashSet<>(pairs); + reducedPairs.remove(pair); + List path = new ArrayList<>(Collections.singleton(pair.src)); + if (isLinked(pair.dst, pair.src, reducedPairs, path) && !in(path, cycles)) { + cycles.add(new ArrayList<>(path)); + log.error("A cycle of length " + (path.size() - 1) + " was found: " + path.toString()); + } + } + + // for now we want to make sure there is no partition movements of the same topic between a pair of consumers. + // the odds of finding a cycle among more than two consumers seem to be very low (according to various randomized + // tests with the given sticky algorithm) that it should not worth the added complexity of handling those cases. + for (List cycle: cycles) + if (cycle.size() == 3) // indicates a cycle of length 2 + return true; + return false; + } + + private boolean isSticky() { + for (Map.Entry>> topicMovements: this.partitionMovementsByTopic.entrySet()) { + Set topicMovementPairs = topicMovements.getValue().keySet(); + if (hasCycles(topicMovementPairs)) { + log.error("Stickiness is violated for topic " + topicMovements.getKey() + + "\nPartition movements for this topic occurred among the following consumer pairs:" + + "\n" + topicMovements.getValue().toString()); + return false; + } + } + + return true; + } + } + + /** + * ConsumerPair represents a pair of Kafka consumer ids involved in a partition reassignment. Each + * ConsumerPair object, which contains a source (src) and a destination (dst) + * element, normally corresponds to a particular partition or topic, and indicates that the particular partition or some + * partition of the particular topic was moved from the source consumer to the destination consumer during the rebalance. + * This class is used, through the PartitionMovements class, by the sticky assignor and helps in determining + * whether a partition reassignment results in cycles among the generated graph of consumer pairs. + */ + private static class ConsumerPair { + private final String src; + private final String dst; + + ConsumerPair(String src, String dst) { + this.src = src; + this.dst = dst; + } + + public String toString() { + return this.src + "->" + this.dst; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((this.src == null) ? 0 : this.src.hashCode()); + result = prime * result + ((this.dst == null) ? 0 : this.dst.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (obj == null) + return false; + + if (!getClass().isInstance(obj)) + return false; + + ConsumerPair otherPair = (ConsumerPair) obj; + return this.src.equals(otherPair.src) && this.dst.equals(otherPair.dst); + } + + private boolean in(Set pairs) { + for (ConsumerPair pair: pairs) + if (this.equals(pair)) + return true; + return false; + } + } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java index 07dd9b87ecab..bf591c07eb5f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerProtocol.java @@ -66,7 +66,7 @@ public class ConsumerProtocol { public static final short CONSUMER_PROTOCOL_V0 = 0; public static final Schema CONSUMER_PROTOCOL_HEADER_SCHEMA = new Schema( new Field(VERSION_KEY_NAME, Type.INT16)); - private static final Struct CONSUMER_PROTOCOL_HEADER_V0 = new Struct(CONSUMER_PROTOCOL_HEADER_SCHEMA) + public static final Struct CONSUMER_PROTOCOL_HEADER_V0 = new Struct(CONSUMER_PROTOCOL_HEADER_SCHEMA) .set(VERSION_KEY_NAME, CONSUMER_PROTOCOL_V0); public static final Schema SUBSCRIPTION_V0 = new Schema( @@ -78,8 +78,6 @@ public class ConsumerProtocol { public static final Schema ASSIGNMENT_V0 = new Schema( new Field(TOPIC_PARTITIONS_KEY_NAME, new ArrayOf(TOPIC_ASSIGNMENT_V0)), new Field(USER_DATA_KEY_NAME, Type.NULLABLE_BYTES)); - public static final Schema TOPIC_PARTITION_ASSIGNMENT_V0 = new Schema( - new Field(TOPIC_PARTITIONS_KEY_NAME, new ArrayOf(TOPIC_ASSIGNMENT_V0))); public static ByteBuffer serializeSubscription(PartitionAssignor.Subscription subscription) { Struct struct = new Struct(SUBSCRIPTION_V0); @@ -140,40 +138,6 @@ public static ByteBuffer serializeAssignment(PartitionAssignor.Assignment assign return buffer; } - public static ByteBuffer serializeTopicPartitionAssignment(List partitions) { - Struct struct = new Struct(TOPIC_PARTITION_ASSIGNMENT_V0); - List topicAssignments = new ArrayList<>(); - for (Map.Entry> topicEntry : asMap(partitions).entrySet()) { - Struct topicAssignment = new Struct(TOPIC_ASSIGNMENT_V0); - topicAssignment.set(TOPIC_KEY_NAME, topicEntry.getKey()); - topicAssignment.set(PARTITIONS_KEY_NAME, topicEntry.getValue().toArray()); - topicAssignments.add(topicAssignment); - } - struct.set(TOPIC_PARTITIONS_KEY_NAME, topicAssignments.toArray()); - ByteBuffer buffer = ByteBuffer.allocate(CONSUMER_PROTOCOL_HEADER_V0.sizeOf() + TOPIC_PARTITION_ASSIGNMENT_V0.sizeOf(struct)); - CONSUMER_PROTOCOL_HEADER_V0.writeTo(buffer); - TOPIC_PARTITION_ASSIGNMENT_V0.write(buffer, struct); - buffer.flip(); - return buffer; - } - - public static List deserializeTopicPartitionAssignment(ByteBuffer buffer) { - Struct header = CONSUMER_PROTOCOL_HEADER_SCHEMA.read(buffer); - Short version = header.getShort(VERSION_KEY_NAME); - checkVersionCompatibility(version); - Struct struct = TOPIC_PARTITION_ASSIGNMENT_V0.read(buffer); - List partitions = new ArrayList<>(); - for (Object structObj : struct.getArray(TOPIC_PARTITIONS_KEY_NAME)) { - Struct assignment = (Struct) structObj; - String topic = assignment.getString(TOPIC_KEY_NAME); - for (Object partitionObj : assignment.getArray(PARTITIONS_KEY_NAME)) { - Integer partition = (Integer) partitionObj; - partitions.add(new TopicPartition(topic, partition)); - } - } - return partitions; - } - private static void checkVersionCompatibility(short version) { // check for invalid versions if (version < CONSUMER_PROTOCOL_V0) @@ -182,8 +146,7 @@ private static void checkVersionCompatibility(short version) { // otherwise, assume versions can be parsed as V0 } - - private static Map> asMap(Collection partitions) { + public static Map> asMap(Collection partitions) { Map> partitionMap = new HashMap<>(); for (TopicPartition partition : partitions) { String topic = partition.topic(); @@ -196,5 +159,4 @@ private static Map> asMap(Collection parti } return partitionMap; } - } diff --git a/clients/src/main/java/org/apache/kafka/common/TopicPartition.java b/clients/src/main/java/org/apache/kafka/common/TopicPartition.java index 832bcd8b21ee..dc79c2e13dc9 100644 --- a/clients/src/main/java/org/apache/kafka/common/TopicPartition.java +++ b/clients/src/main/java/org/apache/kafka/common/TopicPartition.java @@ -75,5 +75,4 @@ public boolean equals(Object obj) { public String toString() { return topic + "-" + partition; } - } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java index 42dc3319cd84..6a3c3f7a4449 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java @@ -21,7 +21,6 @@ import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -31,6 +30,7 @@ import java.util.Random; import java.util.Set; +import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.Utils; import org.junit.Test; @@ -233,17 +233,15 @@ public void testAddRemoveConsumerOneTopic() { String consumer2 = "consumer2"; subscriptions.put(consumer2, topics(topic)); - Map> oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); assertEquals(Arrays.asList(tp(topic, 1), tp(topic, 2)), assignment.get(consumer1)); assertEquals(Arrays.asList(tp(topic, 0)), assignment.get(consumer2)); verifyValidityAndBalance(subscriptions, assignment); assertTrue(isFullyBalanced(assignment)); - assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); + assertTrue(assignor.isSticky()); subscriptions.remove(consumer1); - oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); assertTrue(assignment.get(consumer2).contains(tp(topic, 0))); assertTrue(assignment.get(consumer2).contains(tp(topic, 1))); @@ -251,7 +249,7 @@ public void testAddRemoveConsumerOneTopic() { verifyValidityAndBalance(subscriptions, assignment); assertTrue(isFullyBalanced(assignment)); - assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); + assertTrue(assignor.isSticky()); } /** @@ -315,7 +313,6 @@ public void testAddRemoveTopicTwoConsumers() { partitionsPerTopic.put(topic2, 3); subscriptions.put(consumer1, topics(topic, topic2)); subscriptions.put(consumer2, topics(topic, topic2)); - Map> oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); // verify balance verifyValidityAndBalance(subscriptions, assignment); @@ -326,12 +323,11 @@ public void testAddRemoveTopicTwoConsumers() { assertTrue(consumer1assignment.size() == 3 && consumer2assignment.size() == 3); assertTrue(consumer1assignment.containsAll(consumer1Assignment1)); assertTrue(consumer2assignment.containsAll(consumer2Assignment1)); - assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); + assertTrue(assignor.isSticky()); partitionsPerTopic.remove(topic); subscriptions.put(consumer1, topics(topic2)); subscriptions.put(consumer2, topics(topic2)); - oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); // verify balance verifyValidityAndBalance(subscriptions, assignment); @@ -343,7 +339,7 @@ public void testAddRemoveTopicTwoConsumers() { (consumer1Assignment3.size() == 2 && consumer2Assignment3.size() == 1)); assertTrue(consumer1assignment.containsAll(consumer1Assignment3)); assertTrue(consumer2assignment.containsAll(consumer2Assignment3)); - assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); + assertTrue(assignor.isSticky()); } @Test @@ -364,10 +360,9 @@ public void testReassignmentAfterOneConsumerLeaves() { verifyValidityAndBalance(subscriptions, assignment); subscriptions.remove("consumer10"); - Map> oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); verifyValidityAndBalance(subscriptions, assignment); - assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); + assertTrue(assignor.isSticky()); } @Test @@ -383,10 +378,9 @@ public void testReassignmentAfterOneConsumerAdded() { verifyValidityAndBalance(subscriptions, assignment); subscriptions.put("consumer10", Collections.singletonList("topic")); - Map> oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); verifyValidityAndBalance(subscriptions, assignment); - assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); + assertTrue(assignor.isSticky()); } @Test @@ -407,10 +401,9 @@ public void testSameSubscriptions() { verifyValidityAndBalance(subscriptions, assignment); subscriptions.remove("consumer05"); - Map> oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); verifyValidityAndBalance(subscriptions, assignment); - assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); + assertTrue(assignor.isSticky()); } @Test @@ -439,10 +432,9 @@ public void testLargeAssignmentWithMultipleConsumersLeaving() { subscriptions.remove(c); } - Map> oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); verifyValidityAndBalance(subscriptions, assignment); - assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); + assertTrue(assignor.isSticky()); } @Test @@ -464,10 +456,9 @@ public void testNewSubscription() { subscriptions.get("consumer00").add("topic01"); - Map> oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); verifyValidityAndBalance(subscriptions, assignment); - assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); + assertTrue(assignor.isSticky()); } @Test @@ -507,10 +498,9 @@ public void testReassignmentWithRandomSubscriptionsAndChanges() { subscriptions.put(getConsumerName(i, maxNumConsumers), sub); } - Map> oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); verifyValidityAndBalance(subscriptions, assignment); - assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); + assertTrue(assignor.isSticky()); } } @@ -583,10 +573,9 @@ public void testStickiness() { // removing the potential group leader subscriptions.remove("consumer01"); - Map> oldAssignment = assignor.deepCopy(assignment); assignment = assignor.assign(partitionsPerTopic, subscriptions); verifyValidityAndBalance(subscriptions, assignment); - assertTrue(PartitionMovements.verifyStickiness(oldAssignment, assignment)); + assertTrue(assignor.isSticky()); assignments = assignment.entrySet(); for (Map.Entry> entry: assignments) { @@ -594,7 +583,7 @@ public void testStickiness() { List topicPartitions = entry.getValue(); assertEquals("Consumer " + consumer + " is assigned more topic partitions than expected.", 1, topicPartitions.size()); assertTrue("Stickiness was not honored for consumer " + consumer, - (!partitionsAssigned.containsKey(consumer)) || (assignment.get(consumer).contains(partitionsAssigned.get(consumer)))); + (!partitionsAssigned.containsKey(consumer)) || (assignment.get(consumer).contains(partitionsAssigned.get(consumer)))); } } @@ -676,8 +665,8 @@ private static void verifyValidityAndBalance(Map> subscript if (Math.abs(len - otherLen) <= 1) continue; - Map> map = asMap(partitions); - Map> otherMap = asMap(otherPartitions); + Map> map = ConsumerProtocol.asMap(partitions); + Map> otherMap = ConsumerProtocol.asMap(otherPartitions); if (len > otherLen) { for (String topic: map.keySet()) @@ -697,18 +686,4 @@ private static void verifyValidityAndBalance(Map> subscript } } } - - private static Map> asMap(Collection partitions) { - Map> partitionMap = new HashMap<>(); - for (TopicPartition partition : partitions) { - String topic = partition.topic(); - List topicPartitions = partitionMap.get(topic); - if (topicPartitions == null) { - topicPartitions = new ArrayList<>(); - partitionMap.put(topic, topicPartitions); - } - topicPartitions.add(partition.partition()); - } - return partitionMap; - } } From 3c7d9e66cbdcbc674de45c94e050b39d3f0e8201 Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Tue, 16 May 2017 23:11:49 -0700 Subject: [PATCH 8/9] Minor updates to address feedback * Limited the scope of some classes and members * Removed `StickyAssignor`'s dependency to `ConsumerProtocol` * Simplified the 'previous_assignment' schema used in `StickyAssignor` --- .../clients/consumer/StickyAssignor.java | 108 +++++++++--------- .../clients/consumer/StickyAssignorTest.java | 5 +- 2 files changed, 54 insertions(+), 59 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java index 387cf3574030..86cff1ed51f1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java @@ -32,13 +32,12 @@ import java.util.TreeSet; import org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor; -import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.types.ArrayOf; import org.apache.kafka.common.protocol.types.Field; import org.apache.kafka.common.protocol.types.Schema; -import org.apache.kafka.common.protocol.types.SchemaException; import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.protocol.types.Type; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -175,27 +174,21 @@ public class StickyAssignor extends AbstractPartitionAssignor { private static final Logger log = LoggerFactory.getLogger(StickyAssignor.class); - // this schema is used for preserving consumer's assigned partitions list and - // sending it to the leader during a rebalance - public static final Schema TOPIC_PARTITION_ASSIGNMENT_V0 = new Schema( - new Field(ConsumerProtocol.TOPIC_PARTITIONS_KEY_NAME, new ArrayOf(ConsumerProtocol.TOPIC_ASSIGNMENT_V0))); - - protected Map> currentAssignment = new HashMap<>(); + // these schemas are used for preserving consumer's previously assigned partitions + // list and sending it as user data to the leader during a rebalance + private static final String TOPIC_PARTITIONS_KEY_NAME = "previous_assignment"; + private static final String TOPIC_KEY_NAME = "topic"; + private static final String PARTITIONS_KEY_NAME = "partitions"; + private static final Schema TOPIC_ASSIGNMENT = new Schema( + new Field(TOPIC_KEY_NAME, Type.STRING), + new Field(PARTITIONS_KEY_NAME, new ArrayOf(Type.INT32))); + private static final Schema STICKY_ASSIGNOR_USER_DATA = new Schema( + new Field(TOPIC_PARTITIONS_KEY_NAME, new ArrayOf(TOPIC_ASSIGNMENT))); + + Map> currentAssignment = new HashMap<>(); private List memberAssignment = null; private PartitionMovements partitionMovements; - private void deepCopy(Map> source, Map> dest) { - dest.clear(); - for (Entry> entry: source.entrySet()) - dest.put(entry.getKey(), new ArrayList<>(entry.getValue())); - } - - private Map> deepCopy(Map> assignment) { - Map> copy = new HashMap<>(); - deepCopy(assignment, copy); - return copy; - } - public Map> assign(Map partitionsPerTopic, Map> subscriptions) { partitionMovements = new PartitionMovements(); @@ -664,34 +657,28 @@ public boolean isSticky() { } private static ByteBuffer serializeTopicPartitionAssignment(List partitions) { - Struct struct = new Struct(TOPIC_PARTITION_ASSIGNMENT_V0); + Struct struct = new Struct(STICKY_ASSIGNOR_USER_DATA); List topicAssignments = new ArrayList<>(); - for (Map.Entry> topicEntry : ConsumerProtocol.asMap(partitions).entrySet()) { - Struct topicAssignment = new Struct(ConsumerProtocol.TOPIC_ASSIGNMENT_V0); - topicAssignment.set(ConsumerProtocol.TOPIC_KEY_NAME, topicEntry.getKey()); - topicAssignment.set(ConsumerProtocol.PARTITIONS_KEY_NAME, topicEntry.getValue().toArray()); + for (Map.Entry> topicEntry : asMap(partitions).entrySet()) { + Struct topicAssignment = new Struct(TOPIC_ASSIGNMENT); + topicAssignment.set(TOPIC_KEY_NAME, topicEntry.getKey()); + topicAssignment.set(PARTITIONS_KEY_NAME, topicEntry.getValue().toArray()); topicAssignments.add(topicAssignment); } - struct.set(ConsumerProtocol.TOPIC_PARTITIONS_KEY_NAME, topicAssignments.toArray()); - ByteBuffer buffer = ByteBuffer.allocate(ConsumerProtocol.CONSUMER_PROTOCOL_HEADER_V0.sizeOf() + - TOPIC_PARTITION_ASSIGNMENT_V0.sizeOf(struct)); - ConsumerProtocol.CONSUMER_PROTOCOL_HEADER_V0.writeTo(buffer); - TOPIC_PARTITION_ASSIGNMENT_V0.write(buffer, struct); + struct.set(TOPIC_PARTITIONS_KEY_NAME, topicAssignments.toArray()); + ByteBuffer buffer = ByteBuffer.allocate(STICKY_ASSIGNOR_USER_DATA.sizeOf(struct)); + STICKY_ASSIGNOR_USER_DATA.write(buffer, struct); buffer.flip(); return buffer; } private static List deserializeTopicPartitionAssignment(ByteBuffer buffer) { - Struct header = ConsumerProtocol.CONSUMER_PROTOCOL_HEADER_SCHEMA.read(buffer); - Short version = header.getShort(ConsumerProtocol.VERSION_KEY_NAME); - if (version < ConsumerProtocol.CONSUMER_PROTOCOL_V0) - throw new SchemaException("Unsupported subscription version: " + version); - Struct struct = TOPIC_PARTITION_ASSIGNMENT_V0.read(buffer); + Struct struct = STICKY_ASSIGNOR_USER_DATA.read(buffer); List partitions = new ArrayList<>(); - for (Object structObj : struct.getArray(ConsumerProtocol.TOPIC_PARTITIONS_KEY_NAME)) { + for (Object structObj : struct.getArray(TOPIC_PARTITIONS_KEY_NAME)) { Struct assignment = (Struct) structObj; - String topic = assignment.getString(ConsumerProtocol.TOPIC_KEY_NAME); - for (Object partitionObj : assignment.getArray(ConsumerProtocol.PARTITIONS_KEY_NAME)) { + String topic = assignment.getString(TOPIC_KEY_NAME); + for (Object partitionObj : assignment.getArray(PARTITIONS_KEY_NAME)) { Integer partition = (Integer) partitionObj; partitions.add(new TopicPartition(topic, partition)); } @@ -699,6 +686,32 @@ private static List deserializeTopicPartitionAssignment(ByteBuff return partitions; } + private void deepCopy(Map> source, Map> dest) { + dest.clear(); + for (Entry> entry: source.entrySet()) + dest.put(entry.getKey(), new ArrayList<>(entry.getValue())); + } + + private Map> deepCopy(Map> assignment) { + Map> copy = new HashMap<>(); + deepCopy(assignment, copy); + return copy; + } + + public static Map> asMap(Collection partitions) { + Map> partitionMap = new HashMap<>(); + for (TopicPartition partition : partitions) { + String topic = partition.topic(); + List topicPartitions = partitionMap.get(topic); + if (topicPartitions == null) { + topicPartitions = new ArrayList<>(); + partitionMap.put(topic, topicPartitions); + } + topicPartitions.add(partition.partition()); + } + return partitionMap; + } + private static class PartitionComparator implements Comparator, Serializable { private static final long serialVersionUID = 1L; private Map> map; @@ -719,7 +732,7 @@ public int compare(TopicPartition o1, TopicPartition o2) { } } - protected static class SubscriptionComparator implements Comparator, Serializable { + private static class SubscriptionComparator implements Comparator, Serializable { private static final long serialVersionUID = 1L; private Map> map; @@ -736,23 +749,6 @@ public int compare(String o1, String o2) { } } - protected static class PartitionAssignmentComparator implements Comparator, Serializable { - private static final long serialVersionUID = 1L; - private Map> map; - - PartitionAssignmentComparator(Map> map) { - this.map = map; - } - - @Override - public int compare(String o1, String o2) { - int ret = map.get(o1).size() - map.get(o2).size(); - if (ret == 0) - ret = o1.compareTo(o2); - return ret; - } - } - /** * This class maintains some data structures to simplify lookup of partition movements among consumers. At each point of * time during a partition rebalance it keeps track of partition movements corresponding to each topic, and also possible diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java index 6a3c3f7a4449..ff0ca576e148 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java @@ -30,7 +30,6 @@ import java.util.Random; import java.util.Set; -import org.apache.kafka.clients.consumer.internals.ConsumerProtocol; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.Utils; import org.junit.Test; @@ -665,8 +664,8 @@ private static void verifyValidityAndBalance(Map> subscript if (Math.abs(len - otherLen) <= 1) continue; - Map> map = ConsumerProtocol.asMap(partitions); - Map> otherMap = ConsumerProtocol.asMap(otherPartitions); + Map> map = StickyAssignor.asMap(partitions); + Map> otherMap = StickyAssignor.asMap(otherPartitions); if (len > otherLen) { for (String topic: map.keySet()) From 86bfc4eb5eb1b2ee1cfc564671fffa06864ba435 Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Wed, 17 May 2017 16:55:08 -0700 Subject: [PATCH 9/9] Minor updates based on the feedback received on 5/17 --- .../clients/consumer/StickyAssignor.java | 64 ++++++++----------- .../clients/consumer/StickyAssignorTest.java | 55 ++++++++-------- 2 files changed, 53 insertions(+), 66 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java index 86cff1ed51f1..58e5915be38b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/StickyAssignor.java @@ -38,6 +38,7 @@ import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.types.Type; +import org.apache.kafka.common.utils.CollectionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -279,6 +280,7 @@ private void prepopulateCurrentAssignments() { if (subscriptions == null) return; + currentAssignment.clear(); for (Map.Entry subscriptionEntry : subscriptions.entrySet()) { ByteBuffer userData = subscriptionEntry.getValue().userData(); if (userData != null && userData.hasRemaining()) @@ -652,14 +654,14 @@ private void processPartitionMovement(TopicPartition partition, String newConsum sortedCurrentSubscriptions.add(oldConsumer); } - public boolean isSticky() { + boolean isSticky() { return partitionMovements.isSticky(); } private static ByteBuffer serializeTopicPartitionAssignment(List partitions) { Struct struct = new Struct(STICKY_ASSIGNOR_USER_DATA); List topicAssignments = new ArrayList<>(); - for (Map.Entry> topicEntry : asMap(partitions).entrySet()) { + for (Map.Entry> topicEntry : CollectionUtils.groupDataByTopic(partitions).entrySet()) { Struct topicAssignment = new Struct(TOPIC_ASSIGNMENT); topicAssignment.set(TOPIC_KEY_NAME, topicEntry.getKey()); topicAssignment.set(PARTITIONS_KEY_NAME, topicEntry.getValue().toArray()); @@ -698,20 +700,6 @@ private Map> deepCopy(Map> asMap(Collection partitions) { - Map> partitionMap = new HashMap<>(); - for (TopicPartition partition : partitions) { - String topic = partition.topic(); - List topicPartitions = partitionMap.get(topic); - if (topicPartitions == null) { - topicPartitions = new ArrayList<>(); - partitionMap.put(topic, topicPartitions); - } - topicPartitions.add(partition.partition()); - } - return partitionMap; - } - private static class PartitionComparator implements Comparator, Serializable { private static final long serialVersionUID = 1L; private Map> map; @@ -792,11 +780,11 @@ private void movePartition(TopicPartition partition, String oldConsumer, String if (partitionMovements.containsKey(partition)) { // this partition has previously moved ConsumerPair existingPair = removeMovementRecordOfPartition(partition); - assert existingPair.dst.equals(oldConsumer); - if (!existingPair.src.equals(newConsumer)) { + assert existingPair.dstMemberId.equals(oldConsumer); + if (!existingPair.srcMemberId.equals(newConsumer)) { // the partition is not moving back to its previous consumer // return new ConsumerPair2(existingPair.src, newConsumer); - addPartitionMovementRecord(partition, new ConsumerPair(existingPair.src, newConsumer)); + addPartitionMovementRecord(partition, new ConsumerPair(existingPair.srcMemberId, newConsumer)); } } else addPartitionMovementRecord(partition, pair); @@ -810,8 +798,8 @@ private TopicPartition getTheActualPartitionToBeMoved(TopicPartition partition, if (partitionMovements.containsKey(partition)) { // this partition has previously moved - assert oldConsumer.equals(partitionMovements.get(partition).dst); - oldConsumer = partitionMovements.get(partition).src; + assert oldConsumer.equals(partitionMovements.get(partition).dstMemberId); + oldConsumer = partitionMovements.get(partition).srcMemberId; } Map> partitionMovementsForThisTopic = partitionMovementsByTopic.get(topic); @@ -836,11 +824,11 @@ private boolean isLinked(String src, String dst, Set pairs, List reducedSet = new HashSet<>(pairs); reducedSet.remove(pair); - currentPath.add(pair.src); - return isLinked(pair.dst, dst, reducedSet, currentPath); + currentPath.add(pair.srcMemberId); + return isLinked(pair.dstMemberId, dst, reducedSet, currentPath); } return false; @@ -850,10 +838,8 @@ private boolean in(List cycle, Set> cycles) { List superCycle = new ArrayList<>(cycle); superCycle.remove(superCycle.size() - 1); superCycle.addAll(cycle); - Iterator> it = cycles.iterator(); - while (it.hasNext()) { - List next = it.next(); - if (next.size() == cycle.size() && Collections.indexOfSubList(superCycle, next) != -1) + for (List foundCycle: cycles) { + if (foundCycle.size() == cycle.size() && Collections.indexOfSubList(superCycle, foundCycle) != -1) return true; } return false; @@ -864,8 +850,8 @@ private boolean hasCycles(Set pairs) { for (ConsumerPair pair: pairs) { Set reducedPairs = new HashSet<>(pairs); reducedPairs.remove(pair); - List path = new ArrayList<>(Collections.singleton(pair.src)); - if (isLinked(pair.dst, pair.src, reducedPairs, path) && !in(path, cycles)) { + List path = new ArrayList<>(Collections.singleton(pair.srcMemberId)); + if (isLinked(pair.dstMemberId, pair.srcMemberId, reducedPairs, path) && !in(path, cycles)) { cycles.add(new ArrayList<>(path)); log.error("A cycle of length " + (path.size() - 1) + " was found: " + path.toString()); } @@ -904,24 +890,24 @@ private boolean isSticky() { * whether a partition reassignment results in cycles among the generated graph of consumer pairs. */ private static class ConsumerPair { - private final String src; - private final String dst; + private final String srcMemberId; + private final String dstMemberId; - ConsumerPair(String src, String dst) { - this.src = src; - this.dst = dst; + ConsumerPair(String srcMemberId, String dstMemberId) { + this.srcMemberId = srcMemberId; + this.dstMemberId = dstMemberId; } public String toString() { - return this.src + "->" + this.dst; + return this.srcMemberId + "->" + this.dstMemberId; } @Override public int hashCode() { final int prime = 31; int result = 1; - result = prime * result + ((this.src == null) ? 0 : this.src.hashCode()); - result = prime * result + ((this.dst == null) ? 0 : this.dst.hashCode()); + result = prime * result + ((this.srcMemberId == null) ? 0 : this.srcMemberId.hashCode()); + result = prime * result + ((this.dstMemberId == null) ? 0 : this.dstMemberId.hashCode()); return result; } @@ -934,7 +920,7 @@ public boolean equals(Object obj) { return false; ConsumerPair otherPair = (ConsumerPair) obj; - return this.src.equals(otherPair.src) && this.dst.equals(otherPair.dst); + return this.srcMemberId.equals(otherPair.srcMemberId) && this.dstMemberId.equals(otherPair.dstMemberId); } private boolean in(Set pairs) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java index ff0ca576e148..e9cc8288bd69 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/StickyAssignorTest.java @@ -31,6 +31,7 @@ import java.util.Set; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.CollectionUtils; import org.apache.kafka.common.utils.Utils; import org.junit.Test; @@ -503,29 +504,6 @@ public void testReassignmentWithRandomSubscriptionsAndChanges() { } } - private String getTopicName(int i, int maxNum) { - return getCanonicalName("t", i, maxNum); - } - - private String getConsumerName(int i, int maxNum) { - return getCanonicalName("c", i, maxNum); - } - - private String getCanonicalName(String str, int i, int maxNum) { - return str + pad(i, Integer.toString(maxNum).length()); - } - - private String pad(int num, int digits) { - StringBuilder sb = new StringBuilder(); - int iDigits = Integer.toString(num).length(); - - for (int i = 1; i <= digits - iDigits; ++i) - sb.append("0"); - - sb.append(num); - return sb.toString(); - } - @Test public void testMoveExistingAssignments() { Map partitionsPerTopic = new HashMap<>(); @@ -586,11 +564,34 @@ public void testStickiness() { } } - public static List topics(String... topics) { + private String getTopicName(int i, int maxNum) { + return getCanonicalName("t", i, maxNum); + } + + private String getConsumerName(int i, int maxNum) { + return getCanonicalName("c", i, maxNum); + } + + private String getCanonicalName(String str, int i, int maxNum) { + return str + pad(i, Integer.toString(maxNum).length()); + } + + private String pad(int num, int digits) { + StringBuilder sb = new StringBuilder(); + int iDigits = Integer.toString(num).length(); + + for (int i = 1; i <= digits - iDigits; ++i) + sb.append("0"); + + sb.append(num); + return sb.toString(); + } + + private static List topics(String... topics) { return Arrays.asList(topics); } - public static TopicPartition tp(String topic, int partition) { + private static TopicPartition tp(String topic, int partition) { return new TopicPartition(topic, partition); } @@ -664,8 +665,8 @@ private static void verifyValidityAndBalance(Map> subscript if (Math.abs(len - otherLen) <= 1) continue; - Map> map = StickyAssignor.asMap(partitions); - Map> otherMap = StickyAssignor.asMap(otherPartitions); + Map> map = CollectionUtils.groupDataByTopic(partitions); + Map> otherMap = CollectionUtils.groupDataByTopic(otherPartitions); if (len > otherLen) { for (String topic: map.keySet())