mirror of https://github.com/apache/kafka.git
KAFKA-8179: Part 4, add CooperativeStickyAssignor (#7130)
Splits the existing StickyAssignor logic into an AbstractStickyAssignor class, which is extended by the existing (eager) StickyAssignor and by the new CooperativeStickyAssignor which supports incremental cooperative rebalancing. There is no actual change to the logic -- most methods from StickyAssignor were moved to AbstractStickyAssignor to be shared with CooperativeStickyAssignor, and the abstract MemberData memberData(Subscription) method converts the Subscription to the embedded list of owned partitions for each assignor. The "generation" logic is left in, however this is always Optional.empty() for the CooperativeStickyAssignor as onPartitionsLost should always be called when a generation is missed. Reviewers: Jason Gustafson <jason@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
This commit is contained in:
parent
e23a7182d5
commit
cf32a1a6a0
|
@ -0,0 +1,105 @@
|
||||||
|
/*
|
||||||
|
* 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.Arrays;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.Set;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor;
|
||||||
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A cooperative version of the {@link AbstractStickyAssignor AbstractStickyAssignor}. This follows the same (sticky)
|
||||||
|
* assignment logic as {@link StickyAssignor StickyAssignor} but allows for cooperative rebalancing while the
|
||||||
|
* {@link StickyAssignor StickyAssignor} follows the eager rebalancing protocol. See
|
||||||
|
* {@link ConsumerPartitionAssignor.RebalanceProtocol} for an explanation of the rebalancing protocols.
|
||||||
|
* <p>
|
||||||
|
* Users should prefer this assignor for newer clusters.
|
||||||
|
* <p>
|
||||||
|
* To turn on cooperative rebalancing you must set all your consumers to use this {@code PartitionAssignor},
|
||||||
|
* or implement a custom one that returns {@code RebalanceProtocol.COOPERATIVE} in
|
||||||
|
* {@link CooperativeStickyAssignor#supportedProtocols supportedProtocols()}.
|
||||||
|
* <p>
|
||||||
|
* IMPORTANT: if upgrading from 2.3 or earlier, you must follow a specific upgrade path in order to safely turn on
|
||||||
|
* cooperative rebalancing. See the <a href="https://kafka.apache.org/documentation/#upgrade_240_notable">upgrade guide</a> for details.
|
||||||
|
*/
|
||||||
|
public class CooperativeStickyAssignor extends AbstractStickyAssignor {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String name() {
|
||||||
|
return "cooperative-sticky";
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public List<RebalanceProtocol> supportedProtocols() {
|
||||||
|
return Arrays.asList(RebalanceProtocol.COOPERATIVE, RebalanceProtocol.EAGER);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected MemberData memberData(Subscription subscription) {
|
||||||
|
return new MemberData(subscription.ownedPartitions(), Optional.empty());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, List<TopicPartition>> assign(Map<String, Integer> partitionsPerTopic,
|
||||||
|
Map<String, Subscription> subscriptions) {
|
||||||
|
|
||||||
|
final Map<String, List<TopicPartition>> assignments = super.assign(partitionsPerTopic, subscriptions);
|
||||||
|
adjustAssignment(subscriptions, assignments);
|
||||||
|
return assignments;
|
||||||
|
}
|
||||||
|
|
||||||
|
// Following the cooperative rebalancing protocol requires removing partitions that must first be revoked from the assignment
|
||||||
|
private void adjustAssignment(final Map<String, Subscription> subscriptions,
|
||||||
|
final Map<String, List<TopicPartition>> assignments) {
|
||||||
|
|
||||||
|
Map<TopicPartition, String> allAddedPartitions = new HashMap<>();
|
||||||
|
Set<TopicPartition> allRevokedPartitions = new HashSet<>();
|
||||||
|
|
||||||
|
for (final Map.Entry<String, List<TopicPartition>> entry : assignments.entrySet()) {
|
||||||
|
String consumer = entry.getKey();
|
||||||
|
|
||||||
|
List<TopicPartition> ownedPartitions = subscriptions.get(consumer).ownedPartitions();
|
||||||
|
List<TopicPartition> assignedPartitions = entry.getValue();
|
||||||
|
|
||||||
|
List<TopicPartition> addedPartitions = new ArrayList<>(assignedPartitions);
|
||||||
|
addedPartitions.removeAll(ownedPartitions);
|
||||||
|
for (TopicPartition tp : addedPartitions) {
|
||||||
|
allAddedPartitions.put(tp, consumer);
|
||||||
|
}
|
||||||
|
|
||||||
|
final Set<TopicPartition> revokedPartitions = new HashSet<>(ownedPartitions);
|
||||||
|
revokedPartitions.removeAll(assignedPartitions);
|
||||||
|
allRevokedPartitions.addAll(revokedPartitions);
|
||||||
|
}
|
||||||
|
|
||||||
|
// remove any partitions to be revoked from the current assignment
|
||||||
|
for (TopicPartition tp : allRevokedPartitions) {
|
||||||
|
// if partition is being migrated to another consumer, don't assign it there yet
|
||||||
|
if (allAddedPartitions.containsKey(tp)) {
|
||||||
|
String assignedConsumer = allAddedPartitions.get(tp);
|
||||||
|
assignments.get(assignedConsumer).remove(tp);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -16,7 +16,14 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.clients.consumer;
|
package org.apache.kafka.clients.consumer;
|
||||||
|
|
||||||
import org.apache.kafka.clients.consumer.internals.AbstractPartitionAssignor;
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.Set;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor;
|
||||||
import org.apache.kafka.common.TopicPartition;
|
import org.apache.kafka.common.TopicPartition;
|
||||||
import org.apache.kafka.common.protocol.types.ArrayOf;
|
import org.apache.kafka.common.protocol.types.ArrayOf;
|
||||||
import org.apache.kafka.common.protocol.types.Field;
|
import org.apache.kafka.common.protocol.types.Field;
|
||||||
|
@ -24,25 +31,6 @@ import org.apache.kafka.common.protocol.types.Schema;
|
||||||
import org.apache.kafka.common.protocol.types.Struct;
|
import org.apache.kafka.common.protocol.types.Struct;
|
||||||
import org.apache.kafka.common.protocol.types.Type;
|
import org.apache.kafka.common.protocol.types.Type;
|
||||||
import org.apache.kafka.common.utils.CollectionUtils;
|
import org.apache.kafka.common.utils.CollectionUtils;
|
||||||
import org.slf4j.Logger;
|
|
||||||
import org.slf4j.LoggerFactory;
|
|
||||||
|
|
||||||
import java.io.Serializable;
|
|
||||||
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;
|
|
||||||
import java.util.Iterator;
|
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Map.Entry;
|
|
||||||
import java.util.Optional;
|
|
||||||
import java.util.Set;
|
|
||||||
import java.util.TreeMap;
|
|
||||||
import java.util.TreeSet;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* <p>The sticky assignor serves two purposes. First, it guarantees an assignment that is as balanced as possible, meaning either:
|
* <p>The sticky assignor serves two purposes. First, it guarantees an assignment that is as balanced as possible, meaning either:
|
||||||
|
@ -181,9 +169,11 @@ import java.util.TreeSet;
|
||||||
* Any consumer that uses sticky assignment can leverage this listener like this:
|
* Any consumer that uses sticky assignment can leverage this listener like this:
|
||||||
* <code>consumer.subscribe(topics, new TheNewRebalanceListener());</code>
|
* <code>consumer.subscribe(topics, new TheNewRebalanceListener());</code>
|
||||||
*
|
*
|
||||||
|
* Note that you can leverage the {@link CooperativeStickyAssignor} so that only partitions which are being
|
||||||
|
* reassigned to another consumer will be revoked. That is the preferred assignor for newer cluster. See
|
||||||
|
* {@link ConsumerPartitionAssignor.RebalanceProtocol} for a detailed explanation of cooperative rebalancing.
|
||||||
*/
|
*/
|
||||||
public class StickyAssignor extends AbstractPartitionAssignor {
|
public class StickyAssignor extends AbstractStickyAssignor {
|
||||||
private static final Logger log = LoggerFactory.getLogger(StickyAssignor.class);
|
|
||||||
|
|
||||||
// these schemas are used for preserving consumer's previously assigned partitions
|
// these schemas are used for preserving consumer's previously assigned partitions
|
||||||
// list and sending it as user data to the leader during a rebalance
|
// list and sending it as user data to the leader during a rebalance
|
||||||
|
@ -191,177 +181,22 @@ public class StickyAssignor extends AbstractPartitionAssignor {
|
||||||
static final String TOPIC_KEY_NAME = "topic";
|
static final String TOPIC_KEY_NAME = "topic";
|
||||||
static final String PARTITIONS_KEY_NAME = "partitions";
|
static final String PARTITIONS_KEY_NAME = "partitions";
|
||||||
private static final String GENERATION_KEY_NAME = "generation";
|
private static final String GENERATION_KEY_NAME = "generation";
|
||||||
private static final int DEFAULT_GENERATION = -1;
|
|
||||||
static final Schema TOPIC_ASSIGNMENT = new Schema(
|
static final Schema TOPIC_ASSIGNMENT = new Schema(
|
||||||
new Field(TOPIC_KEY_NAME, Type.STRING),
|
new Field(TOPIC_KEY_NAME, Type.STRING),
|
||||||
new Field(PARTITIONS_KEY_NAME, new ArrayOf(Type.INT32)));
|
new Field(PARTITIONS_KEY_NAME, new ArrayOf(Type.INT32)));
|
||||||
static final Schema STICKY_ASSIGNOR_USER_DATA_V0 = new Schema(
|
static final Schema STICKY_ASSIGNOR_USER_DATA_V0 = new Schema(
|
||||||
new Field(TOPIC_PARTITIONS_KEY_NAME, new ArrayOf(TOPIC_ASSIGNMENT)));
|
new Field(TOPIC_PARTITIONS_KEY_NAME, new ArrayOf(TOPIC_ASSIGNMENT)));
|
||||||
private static final Schema STICKY_ASSIGNOR_USER_DATA_V1 = new Schema(
|
private static final Schema STICKY_ASSIGNOR_USER_DATA_V1 = new Schema(
|
||||||
new Field(TOPIC_PARTITIONS_KEY_NAME, new ArrayOf(TOPIC_ASSIGNMENT)),
|
new Field(TOPIC_PARTITIONS_KEY_NAME, new ArrayOf(TOPIC_ASSIGNMENT)),
|
||||||
new Field(GENERATION_KEY_NAME, Type.INT32));
|
new Field(GENERATION_KEY_NAME, Type.INT32));
|
||||||
|
|
||||||
private List<TopicPartition> memberAssignment = null;
|
private List<TopicPartition> memberAssignment = null;
|
||||||
private PartitionMovements partitionMovements;
|
|
||||||
private int generation = DEFAULT_GENERATION; // consumer group generation
|
private int generation = DEFAULT_GENERATION; // consumer group generation
|
||||||
|
|
||||||
static final class ConsumerUserData {
|
|
||||||
final List<TopicPartition> partitions;
|
|
||||||
final Optional<Integer> generation;
|
|
||||||
ConsumerUserData(List<TopicPartition> partitions, Optional<Integer> generation) {
|
|
||||||
this.partitions = partitions;
|
|
||||||
this.generation = generation;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
static final class ConsumerGenerationPair {
|
|
||||||
final String consumer;
|
|
||||||
final int generation;
|
|
||||||
ConsumerGenerationPair(String consumer, int generation) {
|
|
||||||
this.consumer = consumer;
|
|
||||||
this.generation = generation;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Map<String, List<TopicPartition>> assign(Map<String, Integer> partitionsPerTopic,
|
public String name() {
|
||||||
Map<String, Subscription> subscriptions) {
|
return "sticky";
|
||||||
Map<String, List<TopicPartition>> currentAssignment = new HashMap<>();
|
|
||||||
Map<TopicPartition, ConsumerGenerationPair> prevAssignment = new HashMap<>();
|
|
||||||
partitionMovements = new PartitionMovements();
|
|
||||||
|
|
||||||
prepopulateCurrentAssignments(subscriptions, currentAssignment, prevAssignment);
|
|
||||||
boolean isFreshAssignment = currentAssignment.isEmpty();
|
|
||||||
|
|
||||||
// a mapping of all topic partitions to all consumers that can be assigned to them
|
|
||||||
final Map<TopicPartition, List<String>> partition2AllPotentialConsumers = new HashMap<>();
|
|
||||||
// a mapping of all consumers to all potential topic partitions that can be assigned to them
|
|
||||||
final Map<String, List<TopicPartition>> consumer2AllPotentialPartitions = new HashMap<>();
|
|
||||||
|
|
||||||
// initialize partition2AllPotentialConsumers and consumer2AllPotentialPartitions in the following two for loops
|
|
||||||
for (Entry<String, Integer> entry: partitionsPerTopic.entrySet()) {
|
|
||||||
for (int i = 0; i < entry.getValue(); ++i)
|
|
||||||
partition2AllPotentialConsumers.put(new TopicPartition(entry.getKey(), i), new ArrayList<>());
|
|
||||||
}
|
|
||||||
|
|
||||||
for (Entry<String, Subscription> entry: subscriptions.entrySet()) {
|
|
||||||
String consumerId = entry.getKey();
|
|
||||||
consumer2AllPotentialPartitions.put(consumerId, new ArrayList<>());
|
|
||||||
entry.getValue().topics().stream().filter(topic -> partitionsPerTopic.get(topic) != null).forEach(topic -> {
|
|
||||||
for (int i = 0; i < partitionsPerTopic.get(topic); ++i) {
|
|
||||||
TopicPartition topicPartition = new TopicPartition(topic, i);
|
|
||||||
consumer2AllPotentialPartitions.get(consumerId).add(topicPartition);
|
|
||||||
partition2AllPotentialConsumers.get(topicPartition).add(consumerId);
|
|
||||||
}
|
|
||||||
});
|
|
||||||
|
|
||||||
// add this consumer to currentAssignment (with an empty topic partition assignment) if it does not already exist
|
|
||||||
if (!currentAssignment.containsKey(consumerId))
|
|
||||||
currentAssignment.put(consumerId, new ArrayList<>());
|
|
||||||
}
|
|
||||||
|
|
||||||
// a mapping of partition to current consumer
|
|
||||||
Map<TopicPartition, String> currentPartitionConsumer = new HashMap<>();
|
|
||||||
for (Map.Entry<String, List<TopicPartition>> entry: currentAssignment.entrySet())
|
|
||||||
for (TopicPartition topicPartition: entry.getValue())
|
|
||||||
currentPartitionConsumer.put(topicPartition, entry.getKey());
|
|
||||||
|
|
||||||
List<TopicPartition> sortedPartitions = sortPartitions(
|
|
||||||
currentAssignment, prevAssignment.keySet(), isFreshAssignment, partition2AllPotentialConsumers, consumer2AllPotentialPartitions);
|
|
||||||
|
|
||||||
// all partitions that need to be assigned (initially set to all partitions but adjusted in the following loop)
|
|
||||||
List<TopicPartition> unassignedPartitions = new ArrayList<>(sortedPartitions);
|
|
||||||
for (Iterator<Map.Entry<String, List<TopicPartition>>> it = currentAssignment.entrySet().iterator(); it.hasNext();) {
|
|
||||||
Map.Entry<String, List<TopicPartition>> 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<TopicPartition> 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 if (!subscriptions.get(entry.getKey()).topics().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 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);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// 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<String> sortedCurrentSubscriptions = new TreeSet<>(new SubscriptionComparator(currentAssignment));
|
|
||||||
sortedCurrentSubscriptions.addAll(currentAssignment.keySet());
|
|
||||||
|
|
||||||
balance(currentAssignment, prevAssignment, sortedPartitions, unassignedPartitions, sortedCurrentSubscriptions,
|
|
||||||
consumer2AllPotentialPartitions, partition2AllPotentialConsumers, currentPartitionConsumer);
|
|
||||||
return currentAssignment;
|
|
||||||
}
|
|
||||||
|
|
||||||
private void prepopulateCurrentAssignments(Map<String, Subscription> subscriptions,
|
|
||||||
Map<String, List<TopicPartition>> currentAssignment,
|
|
||||||
Map<TopicPartition, ConsumerGenerationPair> prevAssignment) {
|
|
||||||
// we need to process subscriptions' user data with each consumer's reported generation in mind
|
|
||||||
// higher generations overwrite lower generations in case of a conflict
|
|
||||||
// note that a conflict could exists only if user data is for different generations
|
|
||||||
|
|
||||||
// for each partition we create a sorted map of its consumers by generation
|
|
||||||
Map<TopicPartition, TreeMap<Integer, String>> sortedPartitionConsumersByGeneration = new HashMap<>();
|
|
||||||
for (Map.Entry<String, Subscription> subscriptionEntry: subscriptions.entrySet()) {
|
|
||||||
String consumer = subscriptionEntry.getKey();
|
|
||||||
ByteBuffer userData = subscriptionEntry.getValue().userData();
|
|
||||||
if (userData == null || !userData.hasRemaining()) continue;
|
|
||||||
ConsumerUserData consumerUserData = deserializeTopicPartitionAssignment(userData);
|
|
||||||
|
|
||||||
for (TopicPartition partition: consumerUserData.partitions) {
|
|
||||||
if (sortedPartitionConsumersByGeneration.containsKey(partition)) {
|
|
||||||
Map<Integer, String> consumers = sortedPartitionConsumersByGeneration.get(partition);
|
|
||||||
if (consumerUserData.generation.isPresent() && consumers.containsKey(consumerUserData.generation.get())) {
|
|
||||||
// same partition is assigned to two consumers during the same rebalance.
|
|
||||||
// log a warning and skip this record
|
|
||||||
log.warn("Partition '{}' is assigned to multiple consumers following sticky assignment generation {}.",
|
|
||||||
partition, consumerUserData.generation);
|
|
||||||
} else
|
|
||||||
consumers.put(consumerUserData.generation.orElse(DEFAULT_GENERATION), consumer);
|
|
||||||
} else {
|
|
||||||
TreeMap<Integer, String> sortedConsumers = new TreeMap<>();
|
|
||||||
sortedConsumers.put(consumerUserData.generation.orElse(DEFAULT_GENERATION), consumer);
|
|
||||||
sortedPartitionConsumersByGeneration.put(partition, sortedConsumers);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// prevAssignment holds the prior ConsumerGenerationPair (before current) of each partition
|
|
||||||
// current and previous consumers are the last two consumers of each partition in the above sorted map
|
|
||||||
for (Map.Entry<TopicPartition, TreeMap<Integer, String>> partitionConsumersEntry: sortedPartitionConsumersByGeneration.entrySet()) {
|
|
||||||
TopicPartition partition = partitionConsumersEntry.getKey();
|
|
||||||
TreeMap<Integer, String> consumers = partitionConsumersEntry.getValue();
|
|
||||||
Iterator<Integer> it = consumers.descendingKeySet().iterator();
|
|
||||||
|
|
||||||
// let's process the current (most recent) consumer first
|
|
||||||
String consumer = consumers.get(it.next());
|
|
||||||
currentAssignment.computeIfAbsent(consumer, k -> new ArrayList<>());
|
|
||||||
currentAssignment.get(consumer).add(partition);
|
|
||||||
|
|
||||||
// now update previous assignment if any
|
|
||||||
if (it.hasNext()) {
|
|
||||||
int generation = it.next();
|
|
||||||
prevAssignment.put(partition, new ConsumerGenerationPair(consumers.get(generation), generation));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -375,427 +210,38 @@ public class StickyAssignor extends AbstractPartitionAssignor {
|
||||||
if (memberAssignment == null)
|
if (memberAssignment == null)
|
||||||
return null;
|
return null;
|
||||||
|
|
||||||
return serializeTopicPartitionAssignment(new ConsumerUserData(memberAssignment, Optional.of(generation)));
|
return serializeTopicPartitionAssignment(new MemberData(memberAssignment, Optional.of(generation)));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public String name() {
|
protected MemberData memberData(Subscription subscription) {
|
||||||
return "sticky";
|
ByteBuffer userData = subscription.userData();
|
||||||
}
|
if (userData == null || !userData.hasRemaining()) {
|
||||||
|
return new MemberData(Collections.emptyList(), Optional.empty());
|
||||||
int generation() {
|
|
||||||
return generation;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* determine if the current assignment is a balanced one
|
|
||||||
*
|
|
||||||
* @param currentAssignment: the assignment whose balance needs to be checked
|
|
||||||
* @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 true if the given assignment is balanced; false otherwise
|
|
||||||
*/
|
|
||||||
private boolean isBalanced(Map<String, List<TopicPartition>> currentAssignment,
|
|
||||||
TreeSet<String> sortedCurrentSubscriptions,
|
|
||||||
Map<String, List<TopicPartition>> 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 Map<TopicPartition, String> allPartitions = new HashMap<>();
|
|
||||||
Set<Entry<String, List<TopicPartition>>> assignments = currentAssignment.entrySet();
|
|
||||||
for (Map.Entry<String, List<TopicPartition>> entry: assignments) {
|
|
||||||
List<TopicPartition> topicPartitions = entry.getValue();
|
|
||||||
for (TopicPartition topicPartition: topicPartitions) {
|
|
||||||
if (allPartitions.containsKey(topicPartition))
|
|
||||||
log.error("{} is assigned to more than one consumer.", topicPartition);
|
|
||||||
allPartitions.put(topicPartition, entry.getKey());
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
return deserializeTopicPartitionAssignment(userData);
|
||||||
// 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<TopicPartition> 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<TopicPartition> 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("{} can be moved from consumer {} to consumer {} for a more balanced assignment.",
|
|
||||||
topicPartition, otherConsumer, consumer);
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
return true;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
// visible for testing
|
||||||
* @return the balance score of the given assignment, as the sum of assigned partitions size difference of all consumer pairs.
|
static ByteBuffer serializeTopicPartitionAssignment(MemberData memberData) {
|
||||||
* 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.
|
|
||||||
*/
|
|
||||||
private int getBalanceScore(Map<String, List<TopicPartition>> assignment) {
|
|
||||||
int score = 0;
|
|
||||||
|
|
||||||
Map<String, Integer> consumer2AssignmentSize = new HashMap<>();
|
|
||||||
for (Entry<String, List<TopicPartition>> entry: assignment.entrySet())
|
|
||||||
consumer2AssignmentSize.put(entry.getKey(), entry.getValue().size());
|
|
||||||
|
|
||||||
Iterator<Entry<String, Integer>> it = consumer2AssignmentSize.entrySet().iterator();
|
|
||||||
while (it.hasNext()) {
|
|
||||||
Entry<String, Integer> entry = it.next();
|
|
||||||
int consumerAssignmentSize = entry.getValue();
|
|
||||||
it.remove();
|
|
||||||
for (Entry<String, Integer> otherEntry: consumer2AssignmentSize.entrySet())
|
|
||||||
score += Math.abs(consumerAssignmentSize - otherEntry.getValue());
|
|
||||||
}
|
|
||||||
|
|
||||||
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 currentAssignment the calculated assignment so far
|
|
||||||
* @param partitionsWithADifferentPreviousAssignment partitions that had a different consumer before (for every
|
|
||||||
* such partition there should also be a mapping in
|
|
||||||
* @currentAssignment to a different consumer)
|
|
||||||
* @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<TopicPartition> sortPartitions(Map<String, List<TopicPartition>> currentAssignment,
|
|
||||||
Set<TopicPartition> partitionsWithADifferentPreviousAssignment,
|
|
||||||
boolean isFreshAssignment,
|
|
||||||
Map<TopicPartition, List<String>> partition2AllPotentialConsumers,
|
|
||||||
Map<String, List<TopicPartition>> consumer2AllPotentialPartitions) {
|
|
||||||
List<TopicPartition> 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<String, List<TopicPartition>> assignments = deepCopy(currentAssignment);
|
|
||||||
for (Entry<String, List<TopicPartition>> entry: assignments.entrySet()) {
|
|
||||||
List<TopicPartition> 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<String> sortedConsumers = new TreeSet<>(new SubscriptionComparator(assignments));
|
|
||||||
sortedConsumers.addAll(assignments.keySet());
|
|
||||||
// at this point, sortedConsumers contains an ascending-sorted list of consumers based on
|
|
||||||
// how many valid partitions are currently assigned to them
|
|
||||||
|
|
||||||
while (!sortedConsumers.isEmpty()) {
|
|
||||||
// take the consumer with the most partitions
|
|
||||||
String consumer = sortedConsumers.pollLast();
|
|
||||||
// currently assigned partitions to this consumer
|
|
||||||
List<TopicPartition> remainingPartitions = assignments.get(consumer);
|
|
||||||
// partitions that were assigned to a different consumer last time
|
|
||||||
List<TopicPartition> prevPartitions = new ArrayList<>(partitionsWithADifferentPreviousAssignment);
|
|
||||||
// from partitions that had a different consumer before, keep only those that are
|
|
||||||
// assigned to this consumer now
|
|
||||||
prevPartitions.retainAll(remainingPartitions);
|
|
||||||
if (!prevPartitions.isEmpty()) {
|
|
||||||
// if there is a partition of this consumer that was assigned to another consumer before
|
|
||||||
// mark it as good options for reassignment
|
|
||||||
TopicPartition partition = prevPartitions.remove(0);
|
|
||||||
remainingPartitions.remove(partition);
|
|
||||||
sortedPartitions.add(partition);
|
|
||||||
sortedConsumers.add(consumer);
|
|
||||||
} else if (!remainingPartitions.isEmpty()) {
|
|
||||||
// otherwise, mark any other one of the current partitions as a reassignment candidate
|
|
||||||
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<TopicPartition> 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(Map<TopicPartition, List<String>> partition2AllPotentialConsumers,
|
|
||||||
Map<String, List<TopicPartition>> consumer2AllPotentialPartitions) {
|
|
||||||
if (!hasIdenticalListElements(partition2AllPotentialConsumers.values()))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
return hasIdenticalListElements(consumer2AllPotentialPartitions.values());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* The assignment should improve the overall balance of the partition assignments to consumers.
|
|
||||||
*/
|
|
||||||
private void assignPartition(TopicPartition partition,
|
|
||||||
TreeSet<String> sortedCurrentSubscriptions,
|
|
||||||
Map<String, List<TopicPartition>> currentAssignment,
|
|
||||||
Map<String, List<TopicPartition>> consumer2AllPotentialPartitions,
|
|
||||||
Map<TopicPartition, String> 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);
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private boolean canParticipateInReassignment(TopicPartition partition,
|
|
||||||
Map<TopicPartition, List<String>> 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,
|
|
||||||
Map<String, List<TopicPartition>> currentAssignment,
|
|
||||||
Map<String, List<TopicPartition>> consumer2AllPotentialPartitions,
|
|
||||||
Map<TopicPartition, List<String>> partition2AllPotentialConsumers) {
|
|
||||||
List<TopicPartition> currentPartitions = currentAssignment.get(consumer);
|
|
||||||
int currentAssignmentSize = currentPartitions.size();
|
|
||||||
int maxAssignmentSize = consumer2AllPotentialPartitions.get(consumer).size();
|
|
||||||
if (currentAssignmentSize > maxAssignmentSize)
|
|
||||||
log.error("The consumer {} is assigned more partitions than the maximum possible.", consumer);
|
|
||||||
|
|
||||||
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(Map<String, List<TopicPartition>> currentAssignment,
|
|
||||||
Map<TopicPartition, ConsumerGenerationPair> prevAssignment,
|
|
||||||
List<TopicPartition> sortedPartitions,
|
|
||||||
List<TopicPartition> unassignedPartitions,
|
|
||||||
TreeSet<String> sortedCurrentSubscriptions,
|
|
||||||
Map<String, List<TopicPartition>> consumer2AllPotentialPartitions,
|
|
||||||
Map<TopicPartition, List<String>> partition2AllPotentialConsumers,
|
|
||||||
Map<TopicPartition, String> currentPartitionConsumer) {
|
|
||||||
boolean initializing = currentAssignment.get(sortedCurrentSubscriptions.last()).isEmpty();
|
|
||||||
boolean reassignmentPerformed = false;
|
|
||||||
|
|
||||||
// assign all unassigned partitions
|
|
||||||
for (TopicPartition partition: unassignedPartitions) {
|
|
||||||
// skip if there is no potential consumer for the partition
|
|
||||||
if (partition2AllPotentialConsumers.get(partition).isEmpty())
|
|
||||||
continue;
|
|
||||||
|
|
||||||
assignPartition(partition, sortedCurrentSubscriptions, currentAssignment,
|
|
||||||
consumer2AllPotentialPartitions, currentPartitionConsumer);
|
|
||||||
}
|
|
||||||
|
|
||||||
// narrow down the reassignment scope to only those partitions that can actually be reassigned
|
|
||||||
Set<TopicPartition> fixedPartitions = new HashSet<>();
|
|
||||||
for (TopicPartition partition: partition2AllPotentialConsumers.keySet())
|
|
||||||
if (!canParticipateInReassignment(partition, partition2AllPotentialConsumers))
|
|
||||||
fixedPartitions.add(partition);
|
|
||||||
sortedPartitions.removeAll(fixedPartitions);
|
|
||||||
|
|
||||||
// narrow down the reassignment scope to only those consumers that are subject to reassignment
|
|
||||||
Map<String, List<TopicPartition>> fixedAssignments = new HashMap<>();
|
|
||||||
for (String consumer: consumer2AllPotentialPartitions.keySet())
|
|
||||||
if (!canParticipateInReassignment(consumer, currentAssignment,
|
|
||||||
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<String, List<TopicPartition>> preBalanceAssignment = deepCopy(currentAssignment);
|
|
||||||
Map<TopicPartition, String> preBalancePartitionConsumers = new HashMap<>(currentPartitionConsumer);
|
|
||||||
|
|
||||||
reassignmentPerformed = performReassignments(sortedPartitions, currentAssignment, prevAssignment, 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<String, List<TopicPartition>> entry: fixedAssignments.entrySet()) {
|
|
||||||
String consumer = entry.getKey();
|
|
||||||
currentAssignment.put(consumer, entry.getValue());
|
|
||||||
sortedCurrentSubscriptions.add(consumer);
|
|
||||||
}
|
|
||||||
|
|
||||||
fixedAssignments.clear();
|
|
||||||
}
|
|
||||||
|
|
||||||
private boolean performReassignments(List<TopicPartition> reassignablePartitions,
|
|
||||||
Map<String, List<TopicPartition>> currentAssignment,
|
|
||||||
Map<TopicPartition, ConsumerGenerationPair> prevAssignment,
|
|
||||||
TreeSet<String> sortedCurrentSubscriptions,
|
|
||||||
Map<String, List<TopicPartition>> consumer2AllPotentialPartitions,
|
|
||||||
Map<TopicPartition, List<String>> partition2AllPotentialConsumers,
|
|
||||||
Map<TopicPartition, String> currentPartitionConsumer) {
|
|
||||||
boolean reassignmentPerformed = false;
|
|
||||||
boolean modified;
|
|
||||||
|
|
||||||
// repeat reassignment until no partition can be moved to improve the balance
|
|
||||||
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
|
|
||||||
Iterator<TopicPartition> partitionIterator = reassignablePartitions.iterator();
|
|
||||||
while (partitionIterator.hasNext() && !isBalanced(currentAssignment, 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 '{}' to be assigned to a consumer", partition);
|
|
||||||
|
|
||||||
if (prevAssignment.containsKey(partition) &&
|
|
||||||
currentAssignment.get(consumer).size() > currentAssignment.get(prevAssignment.get(partition).consumer).size() + 1) {
|
|
||||||
reassignPartition(partition, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer, prevAssignment.get(partition).consumer);
|
|
||||||
reassignmentPerformed = true;
|
|
||||||
modified = true;
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
// 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) {
|
|
||||||
reassignPartition(partition, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer, consumer2AllPotentialPartitions);
|
|
||||||
reassignmentPerformed = true;
|
|
||||||
modified = true;
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
} while (modified);
|
|
||||||
|
|
||||||
return reassignmentPerformed;
|
|
||||||
}
|
|
||||||
|
|
||||||
private void reassignPartition(TopicPartition partition,
|
|
||||||
Map<String, List<TopicPartition>> currentAssignment,
|
|
||||||
TreeSet<String> sortedCurrentSubscriptions,
|
|
||||||
Map<TopicPartition, String> currentPartitionConsumer,
|
|
||||||
Map<String, List<TopicPartition>> consumer2AllPotentialPartitions) {
|
|
||||||
// find the new consumer
|
|
||||||
String newConsumer = null;
|
|
||||||
for (String anotherConsumer: sortedCurrentSubscriptions) {
|
|
||||||
if (consumer2AllPotentialPartitions.get(anotherConsumer).contains(partition)) {
|
|
||||||
newConsumer = anotherConsumer;
|
|
||||||
break;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
assert newConsumer != null;
|
|
||||||
|
|
||||||
reassignPartition(partition, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer, newConsumer);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void reassignPartition(TopicPartition partition,
|
|
||||||
Map<String, List<TopicPartition>> currentAssignment,
|
|
||||||
TreeSet<String> sortedCurrentSubscriptions,
|
|
||||||
Map<TopicPartition, String> currentPartitionConsumer,
|
|
||||||
String newConsumer) {
|
|
||||||
String consumer = currentPartitionConsumer.get(partition);
|
|
||||||
// find the correct partition movement considering the stickiness requirement
|
|
||||||
TopicPartition partitionToBeMoved = partitionMovements.getTheActualPartitionToBeMoved(partition, consumer, newConsumer);
|
|
||||||
processPartitionMovement(partitionToBeMoved, newConsumer, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer);
|
|
||||||
}
|
|
||||||
|
|
||||||
private void processPartitionMovement(TopicPartition partition,
|
|
||||||
String newConsumer,
|
|
||||||
Map<String, List<TopicPartition>> currentAssignment,
|
|
||||||
TreeSet<String> sortedCurrentSubscriptions,
|
|
||||||
Map<TopicPartition, String> 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);
|
|
||||||
}
|
|
||||||
|
|
||||||
boolean isSticky() {
|
|
||||||
return partitionMovements.isSticky();
|
|
||||||
}
|
|
||||||
|
|
||||||
static ByteBuffer serializeTopicPartitionAssignment(ConsumerUserData consumerUserData) {
|
|
||||||
Struct struct = new Struct(STICKY_ASSIGNOR_USER_DATA_V1);
|
Struct struct = new Struct(STICKY_ASSIGNOR_USER_DATA_V1);
|
||||||
List<Struct> topicAssignments = new ArrayList<>();
|
List<Struct> topicAssignments = new ArrayList<>();
|
||||||
for (Map.Entry<String, List<Integer>> topicEntry : CollectionUtils.groupPartitionsByTopic(consumerUserData.partitions).entrySet()) {
|
for (Map.Entry<String, List<Integer>> topicEntry : CollectionUtils.groupPartitionsByTopic(memberData.partitions).entrySet()) {
|
||||||
Struct topicAssignment = new Struct(TOPIC_ASSIGNMENT);
|
Struct topicAssignment = new Struct(TOPIC_ASSIGNMENT);
|
||||||
topicAssignment.set(TOPIC_KEY_NAME, topicEntry.getKey());
|
topicAssignment.set(TOPIC_KEY_NAME, topicEntry.getKey());
|
||||||
topicAssignment.set(PARTITIONS_KEY_NAME, topicEntry.getValue().toArray());
|
topicAssignment.set(PARTITIONS_KEY_NAME, topicEntry.getValue().toArray());
|
||||||
topicAssignments.add(topicAssignment);
|
topicAssignments.add(topicAssignment);
|
||||||
}
|
}
|
||||||
struct.set(TOPIC_PARTITIONS_KEY_NAME, topicAssignments.toArray());
|
struct.set(TOPIC_PARTITIONS_KEY_NAME, topicAssignments.toArray());
|
||||||
if (consumerUserData.generation.isPresent())
|
if (memberData.generation.isPresent())
|
||||||
struct.set(GENERATION_KEY_NAME, consumerUserData.generation.get());
|
struct.set(GENERATION_KEY_NAME, memberData.generation.get());
|
||||||
ByteBuffer buffer = ByteBuffer.allocate(STICKY_ASSIGNOR_USER_DATA_V1.sizeOf(struct));
|
ByteBuffer buffer = ByteBuffer.allocate(STICKY_ASSIGNOR_USER_DATA_V1.sizeOf(struct));
|
||||||
STICKY_ASSIGNOR_USER_DATA_V1.write(buffer, struct);
|
STICKY_ASSIGNOR_USER_DATA_V1.write(buffer, struct);
|
||||||
buffer.flip();
|
buffer.flip();
|
||||||
return buffer;
|
return buffer;
|
||||||
}
|
}
|
||||||
|
|
||||||
private static ConsumerUserData deserializeTopicPartitionAssignment(ByteBuffer buffer) {
|
private static MemberData deserializeTopicPartitionAssignment(ByteBuffer buffer) {
|
||||||
Struct struct;
|
Struct struct;
|
||||||
ByteBuffer copy = buffer.duplicate();
|
ByteBuffer copy = buffer.duplicate();
|
||||||
try {
|
try {
|
||||||
|
@ -806,7 +252,7 @@ public class StickyAssignor extends AbstractPartitionAssignor {
|
||||||
struct = STICKY_ASSIGNOR_USER_DATA_V0.read(copy);
|
struct = STICKY_ASSIGNOR_USER_DATA_V0.read(copy);
|
||||||
} catch (Exception e2) {
|
} catch (Exception e2) {
|
||||||
// ignore the consumer's previous assignment if it cannot be parsed
|
// ignore the consumer's previous assignment if it cannot be parsed
|
||||||
return new ConsumerUserData(Collections.emptyList(), Optional.of(DEFAULT_GENERATION));
|
return new MemberData(Collections.emptyList(), Optional.of(DEFAULT_GENERATION));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -821,267 +267,6 @@ public class StickyAssignor extends AbstractPartitionAssignor {
|
||||||
}
|
}
|
||||||
// make sure this is backward compatible
|
// make sure this is backward compatible
|
||||||
Optional<Integer> generation = struct.hasField(GENERATION_KEY_NAME) ? Optional.of(struct.getInt(GENERATION_KEY_NAME)) : Optional.empty();
|
Optional<Integer> generation = struct.hasField(GENERATION_KEY_NAME) ? Optional.of(struct.getInt(GENERATION_KEY_NAME)) : Optional.empty();
|
||||||
return new ConsumerUserData(partitions, generation);
|
return new MemberData(partitions, generation);
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* @param col a collection of elements of type list
|
|
||||||
* @return true if all lists in the collection have the same members; false otherwise
|
|
||||||
*/
|
|
||||||
private <T> boolean hasIdenticalListElements(Collection<List<T>> col) {
|
|
||||||
Iterator<List<T>> it = col.iterator();
|
|
||||||
if (!it.hasNext())
|
|
||||||
return true;
|
|
||||||
List<T> cur = it.next();
|
|
||||||
while (it.hasNext()) {
|
|
||||||
List<T> next = it.next();
|
|
||||||
if (!(cur.containsAll(next) && next.containsAll(cur)))
|
|
||||||
return false;
|
|
||||||
cur = next;
|
|
||||||
}
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
private void deepCopy(Map<String, List<TopicPartition>> source, Map<String, List<TopicPartition>> dest) {
|
|
||||||
dest.clear();
|
|
||||||
for (Entry<String, List<TopicPartition>> entry: source.entrySet())
|
|
||||||
dest.put(entry.getKey(), new ArrayList<>(entry.getValue()));
|
|
||||||
}
|
|
||||||
|
|
||||||
private Map<String, List<TopicPartition>> deepCopy(Map<String, List<TopicPartition>> assignment) {
|
|
||||||
Map<String, List<TopicPartition>> copy = new HashMap<>();
|
|
||||||
deepCopy(assignment, copy);
|
|
||||||
return copy;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class PartitionComparator implements Comparator<TopicPartition>, Serializable {
|
|
||||||
private static final long serialVersionUID = 1L;
|
|
||||||
private Map<TopicPartition, List<String>> map;
|
|
||||||
|
|
||||||
PartitionComparator(Map<TopicPartition, List<String>> 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;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class SubscriptionComparator implements Comparator<String>, Serializable {
|
|
||||||
private static final long serialVersionUID = 1L;
|
|
||||||
private Map<String, List<TopicPartition>> map;
|
|
||||||
|
|
||||||
SubscriptionComparator(Map<String, List<TopicPartition>> 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
|
|
||||||
* movement (in form a <code>ConsumerPair</code> object) for each partition.
|
|
||||||
*/
|
|
||||||
private static class PartitionMovements {
|
|
||||||
private Map<String, Map<ConsumerPair, Set<TopicPartition>>> partitionMovementsByTopic = new HashMap<>();
|
|
||||||
private Map<TopicPartition, ConsumerPair> partitionMovements = new HashMap<>();
|
|
||||||
|
|
||||||
private ConsumerPair removeMovementRecordOfPartition(TopicPartition partition) {
|
|
||||||
ConsumerPair pair = partitionMovements.remove(partition);
|
|
||||||
|
|
||||||
String topic = partition.topic();
|
|
||||||
Map<ConsumerPair, Set<TopicPartition>> 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<ConsumerPair, Set<TopicPartition>> 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.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.srcMemberId, 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).dstMemberId);
|
|
||||||
oldConsumer = partitionMovements.get(partition).srcMemberId;
|
|
||||||
}
|
|
||||||
|
|
||||||
Map<ConsumerPair, Set<TopicPartition>> 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<ConsumerPair> pairs, List<String> 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.srcMemberId.equals(src)) {
|
|
||||||
Set<ConsumerPair> reducedSet = new HashSet<>(pairs);
|
|
||||||
reducedSet.remove(pair);
|
|
||||||
currentPath.add(pair.srcMemberId);
|
|
||||||
return isLinked(pair.dstMemberId, dst, reducedSet, currentPath);
|
|
||||||
}
|
|
||||||
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
private boolean in(List<String> cycle, Set<List<String>> cycles) {
|
|
||||||
List<String> superCycle = new ArrayList<>(cycle);
|
|
||||||
superCycle.remove(superCycle.size() - 1);
|
|
||||||
superCycle.addAll(cycle);
|
|
||||||
for (List<String> foundCycle: cycles) {
|
|
||||||
if (foundCycle.size() == cycle.size() && Collections.indexOfSubList(superCycle, foundCycle) != -1)
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
private boolean hasCycles(Set<ConsumerPair> pairs) {
|
|
||||||
Set<List<String>> cycles = new HashSet<>();
|
|
||||||
for (ConsumerPair pair: pairs) {
|
|
||||||
Set<ConsumerPair> reducedPairs = new HashSet<>(pairs);
|
|
||||||
reducedPairs.remove(pair);
|
|
||||||
List<String> 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 {} was found: {}", path.size() - 1, 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<String> cycle: cycles)
|
|
||||||
if (cycle.size() == 3) // indicates a cycle of length 2
|
|
||||||
return true;
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
private boolean isSticky() {
|
|
||||||
for (Map.Entry<String, Map<ConsumerPair, Set<TopicPartition>>> topicMovements: this.partitionMovementsByTopic.entrySet()) {
|
|
||||||
Set<ConsumerPair> topicMovementPairs = topicMovements.getValue().keySet();
|
|
||||||
if (hasCycles(topicMovementPairs)) {
|
|
||||||
log.error("Stickiness is violated for topic {}"
|
|
||||||
+ "\nPartition movements for this topic occurred among the following consumer pairs:"
|
|
||||||
+ "\n{}", topicMovements.getKey(), topicMovements.getValue().toString());
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* <code>ConsumerPair</code> represents a pair of Kafka consumer ids involved in a partition reassignment. Each
|
|
||||||
* <code>ConsumerPair</code> object, which contains a source (<code>src</code>) and a destination (<code>dst</code>)
|
|
||||||
* 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 <code>PartitionMovements</code> 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 srcMemberId;
|
|
||||||
private final String dstMemberId;
|
|
||||||
|
|
||||||
ConsumerPair(String srcMemberId, String dstMemberId) {
|
|
||||||
this.srcMemberId = srcMemberId;
|
|
||||||
this.dstMemberId = dstMemberId;
|
|
||||||
}
|
|
||||||
|
|
||||||
public String toString() {
|
|
||||||
return this.srcMemberId + "->" + this.dstMemberId;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int hashCode() {
|
|
||||||
final int prime = 31;
|
|
||||||
int result = 1;
|
|
||||||
result = prime * result + ((this.srcMemberId == null) ? 0 : this.srcMemberId.hashCode());
|
|
||||||
result = prime * result + ((this.dstMemberId == null) ? 0 : this.dstMemberId.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.srcMemberId.equals(otherPair.srcMemberId) && this.dstMemberId.equals(otherPair.dstMemberId);
|
|
||||||
}
|
|
||||||
|
|
||||||
private boolean in(Set<ConsumerPair> pairs) {
|
|
||||||
for (ConsumerPair pair: pairs)
|
|
||||||
if (this.equals(pair))
|
|
||||||
return true;
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -0,0 +1,864 @@
|
||||||
|
/*
|
||||||
|
* 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.internals;
|
||||||
|
|
||||||
|
import java.io.Serializable;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collection;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.Comparator;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.Iterator;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Map.Entry;
|
||||||
|
import java.util.Optional;
|
||||||
|
import java.util.Set;
|
||||||
|
import java.util.TreeMap;
|
||||||
|
import java.util.TreeSet;
|
||||||
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
public abstract class AbstractStickyAssignor extends AbstractPartitionAssignor {
|
||||||
|
private static final Logger log = LoggerFactory.getLogger(AbstractStickyAssignor.class);
|
||||||
|
|
||||||
|
public static final int DEFAULT_GENERATION = -1;
|
||||||
|
|
||||||
|
private PartitionMovements partitionMovements;
|
||||||
|
|
||||||
|
static final class ConsumerGenerationPair {
|
||||||
|
final String consumer;
|
||||||
|
final int generation;
|
||||||
|
ConsumerGenerationPair(String consumer, int generation) {
|
||||||
|
this.consumer = consumer;
|
||||||
|
this.generation = generation;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public static final class MemberData {
|
||||||
|
public final List<TopicPartition> partitions;
|
||||||
|
public final Optional<Integer> generation;
|
||||||
|
public MemberData(List<TopicPartition> partitions, Optional<Integer> generation) {
|
||||||
|
this.partitions = partitions;
|
||||||
|
this.generation = generation;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
abstract protected MemberData memberData(Subscription subscription);
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Map<String, List<TopicPartition>> assign(Map<String, Integer> partitionsPerTopic,
|
||||||
|
Map<String, Subscription> subscriptions) {
|
||||||
|
Map<String, List<TopicPartition>> currentAssignment = new HashMap<>();
|
||||||
|
Map<TopicPartition, ConsumerGenerationPair> prevAssignment = new HashMap<>();
|
||||||
|
partitionMovements = new PartitionMovements();
|
||||||
|
|
||||||
|
prepopulateCurrentAssignments(subscriptions, currentAssignment, prevAssignment);
|
||||||
|
boolean isFreshAssignment = currentAssignment.isEmpty();
|
||||||
|
|
||||||
|
// a mapping of all topic partitions to all consumers that can be assigned to them
|
||||||
|
final Map<TopicPartition, List<String>> partition2AllPotentialConsumers = new HashMap<>();
|
||||||
|
// a mapping of all consumers to all potential topic partitions that can be assigned to them
|
||||||
|
final Map<String, List<TopicPartition>> consumer2AllPotentialPartitions = new HashMap<>();
|
||||||
|
|
||||||
|
// initialize partition2AllPotentialConsumers and consumer2AllPotentialPartitions in the following two for loops
|
||||||
|
for (Entry<String, Integer> entry: partitionsPerTopic.entrySet()) {
|
||||||
|
for (int i = 0; i < entry.getValue(); ++i)
|
||||||
|
partition2AllPotentialConsumers.put(new TopicPartition(entry.getKey(), i), new ArrayList<>());
|
||||||
|
}
|
||||||
|
|
||||||
|
for (Entry<String, Subscription> entry: subscriptions.entrySet()) {
|
||||||
|
String consumerId = entry.getKey();
|
||||||
|
consumer2AllPotentialPartitions.put(consumerId, new ArrayList<>());
|
||||||
|
entry.getValue().topics().stream().filter(topic -> partitionsPerTopic.get(topic) != null).forEach(topic -> {
|
||||||
|
for (int i = 0; i < partitionsPerTopic.get(topic); ++i) {
|
||||||
|
TopicPartition topicPartition = new TopicPartition(topic, i);
|
||||||
|
consumer2AllPotentialPartitions.get(consumerId).add(topicPartition);
|
||||||
|
partition2AllPotentialConsumers.get(topicPartition).add(consumerId);
|
||||||
|
}
|
||||||
|
});
|
||||||
|
|
||||||
|
// add this consumer to currentAssignment (with an empty topic partition assignment) if it does not already exist
|
||||||
|
if (!currentAssignment.containsKey(consumerId))
|
||||||
|
currentAssignment.put(consumerId, new ArrayList<>());
|
||||||
|
}
|
||||||
|
|
||||||
|
// a mapping of partition to current consumer
|
||||||
|
Map<TopicPartition, String> currentPartitionConsumer = new HashMap<>();
|
||||||
|
for (Map.Entry<String, List<TopicPartition>> entry: currentAssignment.entrySet())
|
||||||
|
for (TopicPartition topicPartition: entry.getValue())
|
||||||
|
currentPartitionConsumer.put(topicPartition, entry.getKey());
|
||||||
|
|
||||||
|
List<TopicPartition> sortedPartitions = sortPartitions(
|
||||||
|
currentAssignment, prevAssignment.keySet(), isFreshAssignment, partition2AllPotentialConsumers, consumer2AllPotentialPartitions);
|
||||||
|
|
||||||
|
// all partitions that need to be assigned (initially set to all partitions but adjusted in the following loop)
|
||||||
|
List<TopicPartition> unassignedPartitions = new ArrayList<>(sortedPartitions);
|
||||||
|
boolean revocationRequired = false;
|
||||||
|
for (Iterator<Entry<String, List<TopicPartition>>> it = currentAssignment.entrySet().iterator(); it.hasNext();) {
|
||||||
|
Map.Entry<String, List<TopicPartition>> 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<TopicPartition> 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 if (!subscriptions.get(entry.getKey()).topics().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();
|
||||||
|
revocationRequired = true;
|
||||||
|
} else
|
||||||
|
// 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);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// 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<String> sortedCurrentSubscriptions = new TreeSet<>(new SubscriptionComparator(currentAssignment));
|
||||||
|
sortedCurrentSubscriptions.addAll(currentAssignment.keySet());
|
||||||
|
|
||||||
|
balance(currentAssignment, prevAssignment, sortedPartitions, unassignedPartitions, sortedCurrentSubscriptions,
|
||||||
|
consumer2AllPotentialPartitions, partition2AllPotentialConsumers, currentPartitionConsumer, revocationRequired);
|
||||||
|
return currentAssignment;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void prepopulateCurrentAssignments(Map<String, Subscription> subscriptions,
|
||||||
|
Map<String, List<TopicPartition>> currentAssignment,
|
||||||
|
Map<TopicPartition, ConsumerGenerationPair> prevAssignment) {
|
||||||
|
// we need to process subscriptions' user data with each consumer's reported generation in mind
|
||||||
|
// higher generations overwrite lower generations in case of a conflict
|
||||||
|
// note that a conflict could exists only if user data is for different generations
|
||||||
|
|
||||||
|
// for each partition we create a sorted map of its consumers by generation
|
||||||
|
Map<TopicPartition, TreeMap<Integer, String>> sortedPartitionConsumersByGeneration = new HashMap<>();
|
||||||
|
for (Map.Entry<String, Subscription> subscriptionEntry: subscriptions.entrySet()) {
|
||||||
|
String consumer = subscriptionEntry.getKey();
|
||||||
|
MemberData memberData = memberData(subscriptionEntry.getValue());
|
||||||
|
|
||||||
|
for (TopicPartition partition: memberData.partitions) {
|
||||||
|
if (sortedPartitionConsumersByGeneration.containsKey(partition)) {
|
||||||
|
Map<Integer, String> consumers = sortedPartitionConsumersByGeneration.get(partition);
|
||||||
|
if (memberData.generation.isPresent() && consumers.containsKey(memberData.generation.get())) {
|
||||||
|
// same partition is assigned to two consumers during the same rebalance.
|
||||||
|
// log a warning and skip this record
|
||||||
|
log.warn("Partition '{}' is assigned to multiple consumers following sticky assignment generation {}.",
|
||||||
|
partition, memberData.generation);
|
||||||
|
} else
|
||||||
|
consumers.put(memberData.generation.orElse(DEFAULT_GENERATION), consumer);
|
||||||
|
} else {
|
||||||
|
TreeMap<Integer, String> sortedConsumers = new TreeMap<>();
|
||||||
|
sortedConsumers.put(memberData.generation.orElse(DEFAULT_GENERATION), consumer);
|
||||||
|
sortedPartitionConsumersByGeneration.put(partition, sortedConsumers);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// prevAssignment holds the prior ConsumerGenerationPair (before current) of each partition
|
||||||
|
// current and previous consumers are the last two consumers of each partition in the above sorted map
|
||||||
|
for (Map.Entry<TopicPartition, TreeMap<Integer, String>> partitionConsumersEntry: sortedPartitionConsumersByGeneration.entrySet()) {
|
||||||
|
TopicPartition partition = partitionConsumersEntry.getKey();
|
||||||
|
TreeMap<Integer, String> consumers = partitionConsumersEntry.getValue();
|
||||||
|
Iterator<Integer> it = consumers.descendingKeySet().iterator();
|
||||||
|
|
||||||
|
// let's process the current (most recent) consumer first
|
||||||
|
String consumer = consumers.get(it.next());
|
||||||
|
currentAssignment.computeIfAbsent(consumer, k -> new ArrayList<>());
|
||||||
|
currentAssignment.get(consumer).add(partition);
|
||||||
|
|
||||||
|
// now update previous assignment if any
|
||||||
|
if (it.hasNext()) {
|
||||||
|
int generation = it.next();
|
||||||
|
prevAssignment.put(partition, new ConsumerGenerationPair(consumers.get(generation), generation));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* determine if the current assignment is a balanced one
|
||||||
|
*
|
||||||
|
* @param currentAssignment: the assignment whose balance needs to be checked
|
||||||
|
* @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 true if the given assignment is balanced; false otherwise
|
||||||
|
*/
|
||||||
|
private boolean isBalanced(Map<String, List<TopicPartition>> currentAssignment,
|
||||||
|
TreeSet<String> sortedCurrentSubscriptions,
|
||||||
|
Map<String, List<TopicPartition>> 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 Map<TopicPartition, String> allPartitions = new HashMap<>();
|
||||||
|
Set<Entry<String, List<TopicPartition>>> assignments = currentAssignment.entrySet();
|
||||||
|
for (Map.Entry<String, List<TopicPartition>> entry: assignments) {
|
||||||
|
List<TopicPartition> topicPartitions = entry.getValue();
|
||||||
|
for (TopicPartition topicPartition: topicPartitions) {
|
||||||
|
if (allPartitions.containsKey(topicPartition))
|
||||||
|
log.error("{} is assigned to more than one consumer.", topicPartition);
|
||||||
|
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<TopicPartition> 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<TopicPartition> 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("{} can be moved from consumer {} to consumer {} for a more balanced assignment.",
|
||||||
|
topicPartition, otherConsumer, consumer);
|
||||||
|
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.
|
||||||
|
*/
|
||||||
|
private int getBalanceScore(Map<String, List<TopicPartition>> assignment) {
|
||||||
|
int score = 0;
|
||||||
|
|
||||||
|
Map<String, Integer> consumer2AssignmentSize = new HashMap<>();
|
||||||
|
for (Entry<String, List<TopicPartition>> entry: assignment.entrySet())
|
||||||
|
consumer2AssignmentSize.put(entry.getKey(), entry.getValue().size());
|
||||||
|
|
||||||
|
Iterator<Entry<String, Integer>> it = consumer2AssignmentSize.entrySet().iterator();
|
||||||
|
while (it.hasNext()) {
|
||||||
|
Entry<String, Integer> entry = it.next();
|
||||||
|
int consumerAssignmentSize = entry.getValue();
|
||||||
|
it.remove();
|
||||||
|
for (Entry<String, Integer> otherEntry: consumer2AssignmentSize.entrySet())
|
||||||
|
score += Math.abs(consumerAssignmentSize - otherEntry.getValue());
|
||||||
|
}
|
||||||
|
|
||||||
|
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 currentAssignment the calculated assignment so far
|
||||||
|
* @param partitionsWithADifferentPreviousAssignment partitions that had a different consumer before (for every
|
||||||
|
* such partition there should also be a mapping in
|
||||||
|
* @currentAssignment to a different consumer)
|
||||||
|
* @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<TopicPartition> sortPartitions(Map<String, List<TopicPartition>> currentAssignment,
|
||||||
|
Set<TopicPartition> partitionsWithADifferentPreviousAssignment,
|
||||||
|
boolean isFreshAssignment,
|
||||||
|
Map<TopicPartition, List<String>> partition2AllPotentialConsumers,
|
||||||
|
Map<String, List<TopicPartition>> consumer2AllPotentialPartitions) {
|
||||||
|
List<TopicPartition> 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<String, List<TopicPartition>> assignments = deepCopy(currentAssignment);
|
||||||
|
for (Entry<String, List<TopicPartition>> entry: assignments.entrySet()) {
|
||||||
|
List<TopicPartition> 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<String> sortedConsumers = new TreeSet<>(new SubscriptionComparator(assignments));
|
||||||
|
sortedConsumers.addAll(assignments.keySet());
|
||||||
|
// at this point, sortedConsumers contains an ascending-sorted list of consumers based on
|
||||||
|
// how many valid partitions are currently assigned to them
|
||||||
|
|
||||||
|
while (!sortedConsumers.isEmpty()) {
|
||||||
|
// take the consumer with the most partitions
|
||||||
|
String consumer = sortedConsumers.pollLast();
|
||||||
|
// currently assigned partitions to this consumer
|
||||||
|
List<TopicPartition> remainingPartitions = assignments.get(consumer);
|
||||||
|
// partitions that were assigned to a different consumer last time
|
||||||
|
List<TopicPartition> prevPartitions = new ArrayList<>(partitionsWithADifferentPreviousAssignment);
|
||||||
|
// from partitions that had a different consumer before, keep only those that are
|
||||||
|
// assigned to this consumer now
|
||||||
|
prevPartitions.retainAll(remainingPartitions);
|
||||||
|
if (!prevPartitions.isEmpty()) {
|
||||||
|
// if there is a partition of this consumer that was assigned to another consumer before
|
||||||
|
// mark it as good options for reassignment
|
||||||
|
TopicPartition partition = prevPartitions.remove(0);
|
||||||
|
remainingPartitions.remove(partition);
|
||||||
|
sortedPartitions.add(partition);
|
||||||
|
sortedConsumers.add(consumer);
|
||||||
|
} else if (!remainingPartitions.isEmpty()) {
|
||||||
|
// otherwise, mark any other one of the current partitions as a reassignment candidate
|
||||||
|
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<TopicPartition> 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(Map<TopicPartition, List<String>> partition2AllPotentialConsumers,
|
||||||
|
Map<String, List<TopicPartition>> consumer2AllPotentialPartitions) {
|
||||||
|
if (!hasIdenticalListElements(partition2AllPotentialConsumers.values()))
|
||||||
|
return false;
|
||||||
|
|
||||||
|
return hasIdenticalListElements(consumer2AllPotentialPartitions.values());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The assignment should improve the overall balance of the partition assignments to consumers.
|
||||||
|
*/
|
||||||
|
private void assignPartition(TopicPartition partition,
|
||||||
|
TreeSet<String> sortedCurrentSubscriptions,
|
||||||
|
Map<String, List<TopicPartition>> currentAssignment,
|
||||||
|
Map<String, List<TopicPartition>> consumer2AllPotentialPartitions,
|
||||||
|
Map<TopicPartition, String> 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);
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean canParticipateInReassignment(TopicPartition partition,
|
||||||
|
Map<TopicPartition, List<String>> 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,
|
||||||
|
Map<String, List<TopicPartition>> currentAssignment,
|
||||||
|
Map<String, List<TopicPartition>> consumer2AllPotentialPartitions,
|
||||||
|
Map<TopicPartition, List<String>> partition2AllPotentialConsumers) {
|
||||||
|
List<TopicPartition> currentPartitions = currentAssignment.get(consumer);
|
||||||
|
int currentAssignmentSize = currentPartitions.size();
|
||||||
|
int maxAssignmentSize = consumer2AllPotentialPartitions.get(consumer).size();
|
||||||
|
if (currentAssignmentSize > maxAssignmentSize)
|
||||||
|
log.error("The consumer {} is assigned more partitions than the maximum possible.", consumer);
|
||||||
|
|
||||||
|
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(Map<String, List<TopicPartition>> currentAssignment,
|
||||||
|
Map<TopicPartition, ConsumerGenerationPair> prevAssignment,
|
||||||
|
List<TopicPartition> sortedPartitions,
|
||||||
|
List<TopicPartition> unassignedPartitions,
|
||||||
|
TreeSet<String> sortedCurrentSubscriptions,
|
||||||
|
Map<String, List<TopicPartition>> consumer2AllPotentialPartitions,
|
||||||
|
Map<TopicPartition, List<String>> partition2AllPotentialConsumers,
|
||||||
|
Map<TopicPartition, String> currentPartitionConsumer,
|
||||||
|
boolean revocationRequired) {
|
||||||
|
boolean initializing = currentAssignment.get(sortedCurrentSubscriptions.last()).isEmpty();
|
||||||
|
boolean reassignmentPerformed = false;
|
||||||
|
|
||||||
|
// assign all unassigned partitions
|
||||||
|
for (TopicPartition partition: unassignedPartitions) {
|
||||||
|
// skip if there is no potential consumer for the partition
|
||||||
|
if (partition2AllPotentialConsumers.get(partition).isEmpty())
|
||||||
|
continue;
|
||||||
|
|
||||||
|
assignPartition(partition, sortedCurrentSubscriptions, currentAssignment,
|
||||||
|
consumer2AllPotentialPartitions, currentPartitionConsumer);
|
||||||
|
}
|
||||||
|
|
||||||
|
// narrow down the reassignment scope to only those partitions that can actually be reassigned
|
||||||
|
Set<TopicPartition> fixedPartitions = new HashSet<>();
|
||||||
|
for (TopicPartition partition: partition2AllPotentialConsumers.keySet())
|
||||||
|
if (!canParticipateInReassignment(partition, partition2AllPotentialConsumers))
|
||||||
|
fixedPartitions.add(partition);
|
||||||
|
sortedPartitions.removeAll(fixedPartitions);
|
||||||
|
unassignedPartitions.removeAll(fixedPartitions);
|
||||||
|
|
||||||
|
// narrow down the reassignment scope to only those consumers that are subject to reassignment
|
||||||
|
Map<String, List<TopicPartition>> fixedAssignments = new HashMap<>();
|
||||||
|
for (String consumer: consumer2AllPotentialPartitions.keySet())
|
||||||
|
if (!canParticipateInReassignment(consumer, currentAssignment,
|
||||||
|
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<String, List<TopicPartition>> preBalanceAssignment = deepCopy(currentAssignment);
|
||||||
|
Map<TopicPartition, String> preBalancePartitionConsumers = new HashMap<>(currentPartitionConsumer);
|
||||||
|
|
||||||
|
// if we don't already need to revoke something due to subscription changes, first try to balance by only moving newly added partitions
|
||||||
|
if (!revocationRequired) {
|
||||||
|
performReassignments(unassignedPartitions, currentAssignment, prevAssignment, sortedCurrentSubscriptions,
|
||||||
|
consumer2AllPotentialPartitions, partition2AllPotentialConsumers, currentPartitionConsumer);
|
||||||
|
}
|
||||||
|
|
||||||
|
reassignmentPerformed = performReassignments(sortedPartitions, currentAssignment, prevAssignment, 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<String, List<TopicPartition>> entry: fixedAssignments.entrySet()) {
|
||||||
|
String consumer = entry.getKey();
|
||||||
|
currentAssignment.put(consumer, entry.getValue());
|
||||||
|
sortedCurrentSubscriptions.add(consumer);
|
||||||
|
}
|
||||||
|
|
||||||
|
fixedAssignments.clear();
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean performReassignments(List<TopicPartition> reassignablePartitions,
|
||||||
|
Map<String, List<TopicPartition>> currentAssignment,
|
||||||
|
Map<TopicPartition, ConsumerGenerationPair> prevAssignment,
|
||||||
|
TreeSet<String> sortedCurrentSubscriptions,
|
||||||
|
Map<String, List<TopicPartition>> consumer2AllPotentialPartitions,
|
||||||
|
Map<TopicPartition, List<String>> partition2AllPotentialConsumers,
|
||||||
|
Map<TopicPartition, String> currentPartitionConsumer) {
|
||||||
|
boolean reassignmentPerformed = false;
|
||||||
|
boolean modified;
|
||||||
|
|
||||||
|
// repeat reassignment until no partition can be moved to improve the balance
|
||||||
|
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
|
||||||
|
Iterator<TopicPartition> partitionIterator = reassignablePartitions.iterator();
|
||||||
|
while (partitionIterator.hasNext() && !isBalanced(currentAssignment, 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 '{}' to be assigned to a consumer", partition);
|
||||||
|
|
||||||
|
if (prevAssignment.containsKey(partition) &&
|
||||||
|
currentAssignment.get(consumer).size() > currentAssignment.get(prevAssignment.get(partition).consumer).size() + 1) {
|
||||||
|
reassignPartition(partition, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer, prevAssignment.get(partition).consumer);
|
||||||
|
reassignmentPerformed = true;
|
||||||
|
modified = true;
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
// 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) {
|
||||||
|
reassignPartition(partition, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer, consumer2AllPotentialPartitions);
|
||||||
|
reassignmentPerformed = true;
|
||||||
|
modified = true;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} while (modified);
|
||||||
|
|
||||||
|
return reassignmentPerformed;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void reassignPartition(TopicPartition partition,
|
||||||
|
Map<String, List<TopicPartition>> currentAssignment,
|
||||||
|
TreeSet<String> sortedCurrentSubscriptions,
|
||||||
|
Map<TopicPartition, String> currentPartitionConsumer,
|
||||||
|
Map<String, List<TopicPartition>> consumer2AllPotentialPartitions) {
|
||||||
|
// find the new consumer
|
||||||
|
String newConsumer = null;
|
||||||
|
for (String anotherConsumer: sortedCurrentSubscriptions) {
|
||||||
|
if (consumer2AllPotentialPartitions.get(anotherConsumer).contains(partition)) {
|
||||||
|
newConsumer = anotherConsumer;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
assert newConsumer != null;
|
||||||
|
|
||||||
|
reassignPartition(partition, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer, newConsumer);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void reassignPartition(TopicPartition partition,
|
||||||
|
Map<String, List<TopicPartition>> currentAssignment,
|
||||||
|
TreeSet<String> sortedCurrentSubscriptions,
|
||||||
|
Map<TopicPartition, String> currentPartitionConsumer,
|
||||||
|
String newConsumer) {
|
||||||
|
String consumer = currentPartitionConsumer.get(partition);
|
||||||
|
// find the correct partition movement considering the stickiness requirement
|
||||||
|
TopicPartition partitionToBeMoved = partitionMovements.getTheActualPartitionToBeMoved(partition, consumer, newConsumer);
|
||||||
|
processPartitionMovement(partitionToBeMoved, newConsumer, currentAssignment, sortedCurrentSubscriptions, currentPartitionConsumer);
|
||||||
|
}
|
||||||
|
|
||||||
|
private void processPartitionMovement(TopicPartition partition,
|
||||||
|
String newConsumer,
|
||||||
|
Map<String, List<TopicPartition>> currentAssignment,
|
||||||
|
TreeSet<String> sortedCurrentSubscriptions,
|
||||||
|
Map<TopicPartition, String> 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);
|
||||||
|
}
|
||||||
|
|
||||||
|
public boolean isSticky() {
|
||||||
|
return partitionMovements.isSticky();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @param col a collection of elements of type list
|
||||||
|
* @return true if all lists in the collection have the same members; false otherwise
|
||||||
|
*/
|
||||||
|
private <T> boolean hasIdenticalListElements(Collection<List<T>> col) {
|
||||||
|
Iterator<List<T>> it = col.iterator();
|
||||||
|
if (!it.hasNext())
|
||||||
|
return true;
|
||||||
|
List<T> cur = it.next();
|
||||||
|
while (it.hasNext()) {
|
||||||
|
List<T> next = it.next();
|
||||||
|
if (!(cur.containsAll(next) && next.containsAll(cur)))
|
||||||
|
return false;
|
||||||
|
cur = next;
|
||||||
|
}
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
private void deepCopy(Map<String, List<TopicPartition>> source, Map<String, List<TopicPartition>> dest) {
|
||||||
|
dest.clear();
|
||||||
|
for (Entry<String, List<TopicPartition>> entry: source.entrySet())
|
||||||
|
dest.put(entry.getKey(), new ArrayList<>(entry.getValue()));
|
||||||
|
}
|
||||||
|
|
||||||
|
private Map<String, List<TopicPartition>> deepCopy(Map<String, List<TopicPartition>> assignment) {
|
||||||
|
Map<String, List<TopicPartition>> copy = new HashMap<>();
|
||||||
|
deepCopy(assignment, copy);
|
||||||
|
return copy;
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class PartitionComparator implements Comparator<TopicPartition>, Serializable {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
private Map<TopicPartition, List<String>> map;
|
||||||
|
|
||||||
|
PartitionComparator(Map<TopicPartition, List<String>> 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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class SubscriptionComparator implements Comparator<String>, Serializable {
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
private Map<String, List<TopicPartition>> map;
|
||||||
|
|
||||||
|
SubscriptionComparator(Map<String, List<TopicPartition>> 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
|
||||||
|
* movement (in form a <code>ConsumerPair</code> object) for each partition.
|
||||||
|
*/
|
||||||
|
private static class PartitionMovements {
|
||||||
|
private Map<String, Map<ConsumerPair, Set<TopicPartition>>> partitionMovementsByTopic = new HashMap<>();
|
||||||
|
private Map<TopicPartition, ConsumerPair> partitionMovements = new HashMap<>();
|
||||||
|
|
||||||
|
private ConsumerPair removeMovementRecordOfPartition(TopicPartition partition) {
|
||||||
|
ConsumerPair pair = partitionMovements.remove(partition);
|
||||||
|
|
||||||
|
String topic = partition.topic();
|
||||||
|
Map<ConsumerPair, Set<TopicPartition>> 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<ConsumerPair, Set<TopicPartition>> 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.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.srcMemberId, 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).dstMemberId);
|
||||||
|
oldConsumer = partitionMovements.get(partition).srcMemberId;
|
||||||
|
}
|
||||||
|
|
||||||
|
Map<ConsumerPair, Set<TopicPartition>> 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<ConsumerPair> pairs, List<String> 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.srcMemberId.equals(src)) {
|
||||||
|
Set<ConsumerPair> reducedSet = new HashSet<>(pairs);
|
||||||
|
reducedSet.remove(pair);
|
||||||
|
currentPath.add(pair.srcMemberId);
|
||||||
|
return isLinked(pair.dstMemberId, dst, reducedSet, currentPath);
|
||||||
|
}
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean in(List<String> cycle, Set<List<String>> cycles) {
|
||||||
|
List<String> superCycle = new ArrayList<>(cycle);
|
||||||
|
superCycle.remove(superCycle.size() - 1);
|
||||||
|
superCycle.addAll(cycle);
|
||||||
|
for (List<String> foundCycle: cycles) {
|
||||||
|
if (foundCycle.size() == cycle.size() && Collections.indexOfSubList(superCycle, foundCycle) != -1)
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean hasCycles(Set<ConsumerPair> pairs) {
|
||||||
|
Set<List<String>> cycles = new HashSet<>();
|
||||||
|
for (ConsumerPair pair: pairs) {
|
||||||
|
Set<ConsumerPair> reducedPairs = new HashSet<>(pairs);
|
||||||
|
reducedPairs.remove(pair);
|
||||||
|
List<String> 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 {} was found: {}", path.size() - 1, 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<String> cycle: cycles)
|
||||||
|
if (cycle.size() == 3) // indicates a cycle of length 2
|
||||||
|
return true;
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean isSticky() {
|
||||||
|
for (Map.Entry<String, Map<ConsumerPair, Set<TopicPartition>>> topicMovements: this.partitionMovementsByTopic.entrySet()) {
|
||||||
|
Set<ConsumerPair> topicMovementPairs = topicMovements.getValue().keySet();
|
||||||
|
if (hasCycles(topicMovementPairs)) {
|
||||||
|
log.error("Stickiness is violated for topic {}"
|
||||||
|
+ "\nPartition movements for this topic occurred among the following consumer pairs:"
|
||||||
|
+ "\n{}", topicMovements.getKey(), topicMovements.getValue().toString());
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* <code>ConsumerPair</code> represents a pair of Kafka consumer ids involved in a partition reassignment. Each
|
||||||
|
* <code>ConsumerPair</code> object, which contains a source (<code>src</code>) and a destination (<code>dst</code>)
|
||||||
|
* 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 <code>PartitionMovements</code> 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 srcMemberId;
|
||||||
|
private final String dstMemberId;
|
||||||
|
|
||||||
|
ConsumerPair(String srcMemberId, String dstMemberId) {
|
||||||
|
this.srcMemberId = srcMemberId;
|
||||||
|
this.dstMemberId = dstMemberId;
|
||||||
|
}
|
||||||
|
|
||||||
|
public String toString() {
|
||||||
|
return this.srcMemberId + "->" + this.dstMemberId;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int hashCode() {
|
||||||
|
final int prime = 31;
|
||||||
|
int result = 1;
|
||||||
|
result = prime * result + ((this.srcMemberId == null) ? 0 : this.srcMemberId.hashCode());
|
||||||
|
result = prime * result + ((this.dstMemberId == null) ? 0 : this.dstMemberId.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.srcMemberId.equals(otherPair.srcMemberId) && this.dstMemberId.equals(otherPair.dstMemberId);
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean in(Set<ConsumerPair> pairs) {
|
||||||
|
for (ConsumerPair pair: pairs)
|
||||||
|
if (this.equals(pair))
|
||||||
|
return true;
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -38,7 +38,10 @@ import java.util.Set;
|
||||||
* can use this user data to forward the rackId belonging to each member.
|
* can use this user data to forward the rackId belonging to each member.
|
||||||
*
|
*
|
||||||
* This interface has been deprecated in 2.4, custom assignors should now implement
|
* This interface has been deprecated in 2.4, custom assignors should now implement
|
||||||
* {@link org.apache.kafka.clients.consumer.ConsumerPartitionAssignor}
|
* {@link org.apache.kafka.clients.consumer.ConsumerPartitionAssignor}. Note that maintaining compatibility
|
||||||
|
* for an internal interface here is a special case, as {@code PartitionAssignor} was meant to be a public API
|
||||||
|
* although it was placed in the internals package. Users should not expect internal interfaces or classes to
|
||||||
|
* not be removed or maintain compatibility in any way.
|
||||||
*/
|
*/
|
||||||
@Deprecated
|
@Deprecated
|
||||||
public interface PartitionAssignor {
|
public interface PartitionAssignor {
|
||||||
|
|
|
@ -33,7 +33,11 @@ import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This adapter class is used to ensure backwards compatibility for those who have implemented the {@link PartitionAssignor}
|
* This adapter class is used to ensure backwards compatibility for those who have implemented the {@link PartitionAssignor}
|
||||||
* interface, which has been deprecated in favor of the new {@link org.apache.kafka.clients.consumer.ConsumerPartitionAssignor}
|
* interface, which has been deprecated in favor of the new {@link org.apache.kafka.clients.consumer.ConsumerPartitionAssignor}.
|
||||||
|
* <p>
|
||||||
|
* Note that maintaining compatibility for an internal interface here is a special case, as {@code PartitionAssignor}
|
||||||
|
* was meant to be a public API although it was placed in the internals package. Users should not expect internal
|
||||||
|
* interfaces or classes to not be removed or maintain compatibility in any way.
|
||||||
*/
|
*/
|
||||||
@SuppressWarnings("deprecation")
|
@SuppressWarnings("deprecation")
|
||||||
public class PartitionAssignorAdapter implements ConsumerPartitionAssignor {
|
public class PartitionAssignorAdapter implements ConsumerPartitionAssignor {
|
||||||
|
|
|
@ -0,0 +1,105 @@
|
||||||
|
/*
|
||||||
|
* 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.assertTrue;
|
||||||
|
|
||||||
|
import java.util.HashSet;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.AbstractStickyAssignorTest;
|
||||||
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
|
||||||
|
public class CooperativeStickyAssignorTest extends AbstractStickyAssignorTest {
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public AbstractStickyAssignor createAssignor() {
|
||||||
|
return new CooperativeStickyAssignor();
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Subscription buildSubscription(List<String> topics, List<TopicPartition> partitions) {
|
||||||
|
return new Subscription(topics, assignor.subscriptionUserData(new HashSet<>(topics)), partitions);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The cooperative assignor must do some additional work and verification of some assignments relative to the eager
|
||||||
|
* assignor, since it may or may not need to trigger a second follow-up rebalance.
|
||||||
|
* <p>
|
||||||
|
* In addition to the validity requirements described in
|
||||||
|
* {@link org.apache.kafka.clients.consumer.internals.AbstractStickyAssignorTest#verifyValidityAndBalance(Map, Map, Map)},
|
||||||
|
* we must verify that no partition is being revoked and reassigned during the same rebalance. This means the initial
|
||||||
|
* assignment may be unbalanced, so if we do detect partitions being revoked we should trigger a second "rebalance"
|
||||||
|
* to get the final assignment and then verify that it is both valid and balanced.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void verifyValidityAndBalance(Map<String, Subscription> subscriptions,
|
||||||
|
Map<String, List<TopicPartition>> assignments,
|
||||||
|
Map<String, Integer> partitionsPerTopic) {
|
||||||
|
int rebalances = 0;
|
||||||
|
// partitions are being revoked, we must go through another assignment to get the final state
|
||||||
|
while (verifyCooperativeValidity(subscriptions, assignments)) {
|
||||||
|
|
||||||
|
// update the subscriptions with the now owned partitions
|
||||||
|
for (Map.Entry<String, List<TopicPartition>> entry : assignments.entrySet()) {
|
||||||
|
String consumer = entry.getKey();
|
||||||
|
Subscription oldSubscription = subscriptions.get(consumer);
|
||||||
|
subscriptions.put(consumer, buildSubscription(oldSubscription.topics(), entry.getValue()));
|
||||||
|
}
|
||||||
|
|
||||||
|
assignments.clear();
|
||||||
|
assignments.putAll(assignor.assign(partitionsPerTopic, subscriptions));
|
||||||
|
++rebalances;
|
||||||
|
|
||||||
|
assertTrue(rebalances <= 2);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Check the validity and balance of the final assignment
|
||||||
|
super.verifyValidityAndBalance(subscriptions, assignments, partitionsPerTopic);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Returns true if partitions are being revoked, indicating a second rebalance will be triggered
|
||||||
|
private boolean verifyCooperativeValidity(Map<String, Subscription> subscriptions, Map<String, List<TopicPartition>> assignments) {
|
||||||
|
Set<TopicPartition> allAddedPartitions = new HashSet<>();
|
||||||
|
Set<TopicPartition> allRevokedPartitions = new HashSet<>();
|
||||||
|
for (Map.Entry<String, List<TopicPartition>> entry : assignments.entrySet()) {
|
||||||
|
List<TopicPartition> ownedPartitions = subscriptions.get(entry.getKey()).ownedPartitions();
|
||||||
|
List<TopicPartition> assignedPartitions = entry.getValue();
|
||||||
|
|
||||||
|
Set<TopicPartition> revokedPartitions = new HashSet<>(ownedPartitions);
|
||||||
|
revokedPartitions.removeAll(assignedPartitions);
|
||||||
|
|
||||||
|
Set<TopicPartition> addedPartitions = new HashSet<>(assignedPartitions);
|
||||||
|
addedPartitions.removeAll(ownedPartitions);
|
||||||
|
|
||||||
|
allAddedPartitions.addAll(addedPartitions);
|
||||||
|
allRevokedPartitions.addAll(revokedPartitions);
|
||||||
|
}
|
||||||
|
|
||||||
|
Set<TopicPartition> intersection = new HashSet<>(allAddedPartitions);
|
||||||
|
intersection.retainAll(allRevokedPartitions);
|
||||||
|
assertTrue("Error: Some partitions were assigned to a new consumer during the same rebalance they are being "
|
||||||
|
+ "revoked from their previous owner."
|
||||||
|
+ "Partitions: " + intersection.toString(),
|
||||||
|
intersection.isEmpty());
|
||||||
|
|
||||||
|
return !allRevokedPartitions.isEmpty();
|
||||||
|
}
|
||||||
|
}
|
|
@ -16,633 +16,39 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.kafka.clients.consumer;
|
package org.apache.kafka.clients.consumer;
|
||||||
|
|
||||||
import java.nio.ByteBuffer;
|
import static org.apache.kafka.clients.consumer.StickyAssignor.serializeTopicPartitionAssignment;
|
||||||
import java.util.ArrayList;
|
import static org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.DEFAULT_GENERATION;
|
||||||
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.Optional;
|
|
||||||
import java.util.Random;
|
|
||||||
import java.util.Set;
|
|
||||||
|
|
||||||
import org.apache.kafka.clients.consumer.StickyAssignor.ConsumerUserData;
|
|
||||||
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription;
|
|
||||||
import org.apache.kafka.common.TopicPartition;
|
|
||||||
import org.apache.kafka.common.protocol.types.Struct;
|
|
||||||
import org.apache.kafka.common.utils.CollectionUtils;
|
|
||||||
import org.apache.kafka.common.utils.Utils;
|
|
||||||
import org.junit.Before;
|
|
||||||
import org.junit.Test;
|
|
||||||
|
|
||||||
import static org.junit.Assert.assertEquals;
|
import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertFalse;
|
import static org.junit.Assert.assertFalse;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
public class StickyAssignorTest {
|
import java.nio.ByteBuffer;
|
||||||
|
import java.util.ArrayList;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.HashMap;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Optional;
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.AbstractStickyAssignor.MemberData;
|
||||||
|
import org.apache.kafka.clients.consumer.internals.AbstractStickyAssignorTest;
|
||||||
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.protocol.types.Struct;
|
||||||
|
import org.apache.kafka.common.utils.CollectionUtils;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
private StickyAssignor assignor = new StickyAssignor();
|
public class StickyAssignorTest extends AbstractStickyAssignorTest {
|
||||||
private String consumerId = "consumer";
|
|
||||||
private Map<String, Subscription> subscriptions;
|
|
||||||
private String topic = "topic";
|
|
||||||
|
|
||||||
@Before
|
@Override
|
||||||
public void setUp() {
|
public AbstractStickyAssignor createAssignor() {
|
||||||
if (subscriptions != null) {
|
return new StickyAssignor();
|
||||||
subscriptions.clear();
|
|
||||||
} else {
|
|
||||||
subscriptions = new HashMap<>();
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Override
|
||||||
public void testOneConsumerNoTopic() {
|
public Subscription buildSubscription(List<String> topics, List<TopicPartition> partitions) {
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
return new Subscription(topics,
|
||||||
subscriptions = Collections.singletonMap(consumerId, new Subscription(Collections.emptyList()));
|
serializeTopicPartitionAssignment(new MemberData(partitions, Optional.of(DEFAULT_GENERATION))));
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
assertEquals(Collections.singleton(consumerId), assignment.keySet());
|
|
||||||
assertTrue(assignment.get(consumerId).isEmpty());
|
|
||||||
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testOneConsumerNonexistentTopic() {
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
partitionsPerTopic.put(topic, 0);
|
|
||||||
subscriptions = Collections.singletonMap(consumerId, new Subscription(topics(topic)));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
|
|
||||||
assertEquals(Collections.singleton(consumerId), assignment.keySet());
|
|
||||||
assertTrue(assignment.get(consumerId).isEmpty());
|
|
||||||
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testOneConsumerOneTopic() {
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
partitionsPerTopic.put(topic, 3);
|
|
||||||
subscriptions = Collections.singletonMap(consumerId, new Subscription(topics(topic)));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
assertEquals(partitions(tp(topic, 0), tp(topic, 1), tp(topic, 2)), assignment.get(consumerId));
|
|
||||||
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testOnlyAssignsPartitionsFromSubscribedTopics() {
|
|
||||||
String otherTopic = "other";
|
|
||||||
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
partitionsPerTopic.put(topic, 3);
|
|
||||||
partitionsPerTopic.put(otherTopic, 3);
|
|
||||||
subscriptions = Collections.singletonMap(consumerId, new Subscription(topics(topic)));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
assertEquals(partitions(tp(topic, 0), tp(topic, 1), tp(topic, 2)), assignment.get(consumerId));
|
|
||||||
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testOneConsumerMultipleTopics() {
|
|
||||||
String topic1 = "topic1";
|
|
||||||
String topic2 = "topic2";
|
|
||||||
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
partitionsPerTopic.put(topic1, 1);
|
|
||||||
partitionsPerTopic.put(topic2, 2);
|
|
||||||
subscriptions = Collections.singletonMap(consumerId, new Subscription(topics(topic1, topic2)));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
assertEquals(partitions(tp(topic1, 0), tp(topic2, 0), tp(topic2, 1)), assignment.get(consumerId));
|
|
||||||
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testTwoConsumersOneTopicOnePartition() {
|
|
||||||
String consumer1 = "consumer1";
|
|
||||||
String consumer2 = "consumer2";
|
|
||||||
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
partitionsPerTopic.put(topic, 1);
|
|
||||||
|
|
||||||
subscriptions.put(consumer1, new Subscription(topics(topic)));
|
|
||||||
subscriptions.put(consumer2, new Subscription(topics(topic)));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
assertEquals(partitions(tp(topic, 0)), assignment.get(consumer1));
|
|
||||||
assertEquals(Collections.<TopicPartition>emptyList(), assignment.get(consumer2));
|
|
||||||
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testTwoConsumersOneTopicTwoPartitions() {
|
|
||||||
String consumer1 = "consumer1";
|
|
||||||
String consumer2 = "consumer2";
|
|
||||||
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
partitionsPerTopic.put(topic, 2);
|
|
||||||
|
|
||||||
subscriptions.put(consumer1, new Subscription(topics(topic)));
|
|
||||||
subscriptions.put(consumer2, new Subscription(topics(topic)));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
assertEquals(partitions(tp(topic, 0)), assignment.get(consumer1));
|
|
||||||
assertEquals(partitions(tp(topic, 1)), assignment.get(consumer2));
|
|
||||||
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testMultipleConsumersMixedTopicSubscriptions() {
|
|
||||||
String topic1 = "topic1";
|
|
||||||
String topic2 = "topic2";
|
|
||||||
String consumer1 = "consumer1";
|
|
||||||
String consumer2 = "consumer2";
|
|
||||||
String consumer3 = "consumer3";
|
|
||||||
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
partitionsPerTopic.put(topic1, 3);
|
|
||||||
partitionsPerTopic.put(topic2, 2);
|
|
||||||
|
|
||||||
subscriptions.put(consumer1, new Subscription(topics(topic1)));
|
|
||||||
subscriptions.put(consumer2, new Subscription(topics(topic1, topic2)));
|
|
||||||
subscriptions.put(consumer3, new Subscription(topics(topic1)));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
assertEquals(partitions(tp(topic1, 0), tp(topic1, 2)), assignment.get(consumer1));
|
|
||||||
assertEquals(partitions(tp(topic2, 0), tp(topic2, 1)), assignment.get(consumer2));
|
|
||||||
assertEquals(partitions(tp(topic1, 1)), assignment.get(consumer3));
|
|
||||||
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testTwoConsumersTwoTopicsSixPartitions() {
|
|
||||||
String topic1 = "topic1";
|
|
||||||
String topic2 = "topic2";
|
|
||||||
String consumer1 = "consumer1";
|
|
||||||
String consumer2 = "consumer2";
|
|
||||||
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
partitionsPerTopic.put(topic1, 3);
|
|
||||||
partitionsPerTopic.put(topic2, 3);
|
|
||||||
|
|
||||||
subscriptions.put(consumer1, new Subscription(topics(topic1, topic2)));
|
|
||||||
subscriptions.put(consumer2, new Subscription(topics(topic1, topic2)));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
assertEquals(partitions(tp(topic1, 0), tp(topic1, 2), tp(topic2, 1)), assignment.get(consumer1));
|
|
||||||
assertEquals(partitions(tp(topic1, 1), tp(topic2, 0), tp(topic2, 2)), assignment.get(consumer2));
|
|
||||||
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testAddRemoveConsumerOneTopic() {
|
|
||||||
String consumer1 = "consumer1";
|
|
||||||
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
partitionsPerTopic.put(topic, 3);
|
|
||||||
subscriptions.put(consumer1, new Subscription(topics(topic)));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
assertEquals(partitions(tp(topic, 0), tp(topic, 1), tp(topic, 2)), assignment.get(consumer1));
|
|
||||||
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
|
|
||||||
String consumer2 = "consumer2";
|
|
||||||
subscriptions.put(consumer1,
|
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(assignment.get(consumer1), Optional.of(assignor.generation())))));
|
|
||||||
subscriptions.put(consumer2, new Subscription(topics(topic)));
|
|
||||||
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
assertEquals(partitions(tp(topic, 2), tp(topic, 1)), assignment.get(consumer1));
|
|
||||||
assertEquals(partitions(tp(topic, 0)), assignment.get(consumer2));
|
|
||||||
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
assertTrue(assignor.isSticky());
|
|
||||||
|
|
||||||
subscriptions.remove(consumer1);
|
|
||||||
subscriptions.put(consumer2,
|
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(assignment.get(consumer2), Optional.of(assignor.generation())))));
|
|
||||||
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
assertEquals(new HashSet<>(partitions(tp(topic, 2), tp(topic, 1), tp(topic, 0))),
|
|
||||||
new HashSet<>(assignment.get(consumer2)));
|
|
||||||
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
assertTrue(assignor.isSticky());
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* 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<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
for (int i = 1; i <= 5; i++)
|
|
||||||
partitionsPerTopic.put(String.format("topic%d", i), (i % 2) + 1);
|
|
||||||
|
|
||||||
subscriptions.put("consumer1",
|
|
||||||
new Subscription(topics("topic1", "topic2", "topic3", "topic4", "topic5")));
|
|
||||||
subscriptions.put("consumer2",
|
|
||||||
new Subscription(topics("topic1", "topic3", "topic5")));
|
|
||||||
subscriptions.put("consumer3",
|
|
||||||
new Subscription(topics("topic1", "topic3", "topic5")));
|
|
||||||
subscriptions.put("consumer4",
|
|
||||||
new Subscription(topics("topic1", "topic2", "topic3", "topic4", "topic5")));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testAddRemoveTopicTwoConsumers() {
|
|
||||||
String consumer1 = "consumer";
|
|
||||||
String consumer2 = "consumer2";
|
|
||||||
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
partitionsPerTopic.put(topic, 3);
|
|
||||||
subscriptions.put(consumer1, new Subscription(topics(topic)));
|
|
||||||
subscriptions.put(consumer2, new Subscription(topics(topic)));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
// verify balance
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
// verify stickiness
|
|
||||||
List<TopicPartition> consumer1Assignment1 = assignment.get(consumer1);
|
|
||||||
List<TopicPartition> 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,
|
|
||||||
new Subscription(topics(topic, topic2), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(assignment.get(consumer1), Optional.of(assignor.generation())))));
|
|
||||||
subscriptions.put(consumer2,
|
|
||||||
new Subscription(topics(topic, topic2), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(assignment.get(consumer2), Optional.of(assignor.generation())))));
|
|
||||||
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
// verify balance
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
// verify stickiness
|
|
||||||
List<TopicPartition> consumer1assignment = assignment.get(consumer1);
|
|
||||||
List<TopicPartition> consumer2assignment = assignment.get(consumer2);
|
|
||||||
assertTrue(consumer1assignment.size() == 3 && consumer2assignment.size() == 3);
|
|
||||||
assertTrue(consumer1assignment.containsAll(consumer1Assignment1));
|
|
||||||
assertTrue(consumer2assignment.containsAll(consumer2Assignment1));
|
|
||||||
assertTrue(assignor.isSticky());
|
|
||||||
|
|
||||||
partitionsPerTopic.remove(topic);
|
|
||||||
subscriptions.put(consumer1,
|
|
||||||
new Subscription(topics(topic2), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(assignment.get(consumer1), Optional.of(assignor.generation())))));
|
|
||||||
subscriptions.put(consumer2,
|
|
||||||
new Subscription(topics(topic2), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(assignment.get(consumer2), Optional.of(assignor.generation())))));
|
|
||||||
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
// verify balance
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
// verify stickiness
|
|
||||||
List<TopicPartition> consumer1Assignment3 = assignment.get(consumer1);
|
|
||||||
List<TopicPartition> consumer2Assignment3 = assignment.get(consumer2);
|
|
||||||
assertTrue((consumer1Assignment3.size() == 1 && consumer2Assignment3.size() == 2) ||
|
|
||||||
(consumer1Assignment3.size() == 2 && consumer2Assignment3.size() == 1));
|
|
||||||
assertTrue(consumer1assignment.containsAll(consumer1Assignment3));
|
|
||||||
assertTrue(consumer2assignment.containsAll(consumer2Assignment3));
|
|
||||||
assertTrue(assignor.isSticky());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testReassignmentAfterOneConsumerLeaves() {
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
for (int i = 1; i < 20; i++)
|
|
||||||
partitionsPerTopic.put(getTopicName(i, 20), i);
|
|
||||||
|
|
||||||
for (int i = 1; i < 20; i++) {
|
|
||||||
List<String> topics = new ArrayList<>();
|
|
||||||
for (int j = 1; j <= i; j++)
|
|
||||||
topics.add(getTopicName(j, 20));
|
|
||||||
subscriptions.put(getConsumerName(i, 20), new Subscription(topics));
|
|
||||||
}
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
|
|
||||||
for (int i = 1; i < 20; i++) {
|
|
||||||
String consumer = getConsumerName(i, 20);
|
|
||||||
subscriptions.put(consumer,
|
|
||||||
new Subscription(subscriptions.get(consumer).topics(),
|
|
||||||
StickyAssignor.serializeTopicPartitionAssignment(new ConsumerUserData(assignment.get(consumer), Optional.of(assignor.generation())))));
|
|
||||||
}
|
|
||||||
subscriptions.remove("consumer10");
|
|
||||||
|
|
||||||
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(assignor.isSticky());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testReassignmentAfterOneConsumerAdded() {
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
partitionsPerTopic.put("topic", 20);
|
|
||||||
|
|
||||||
for (int i = 1; i < 10; i++)
|
|
||||||
subscriptions.put(getConsumerName(i, 10),
|
|
||||||
new Subscription(topics("topic")));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
|
|
||||||
// add a new consumer
|
|
||||||
subscriptions.put(getConsumerName(10, 10), new Subscription(topics("topic")));
|
|
||||||
|
|
||||||
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(assignor.isSticky());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testSameSubscriptions() {
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
for (int i = 1; i < 15; i++)
|
|
||||||
partitionsPerTopic.put(getTopicName(i, 15), i);
|
|
||||||
|
|
||||||
for (int i = 1; i < 9; i++) {
|
|
||||||
List<String> topics = new ArrayList<>();
|
|
||||||
for (int j = 1; j <= partitionsPerTopic.size(); j++)
|
|
||||||
topics.add(getTopicName(j, 15));
|
|
||||||
subscriptions.put(getConsumerName(i, 9), new Subscription(topics));
|
|
||||||
}
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
|
|
||||||
for (int i = 1; i < 9; i++) {
|
|
||||||
String consumer = getConsumerName(i, 9);
|
|
||||||
subscriptions.put(consumer,
|
|
||||||
new Subscription(subscriptions.get(consumer).topics(),
|
|
||||||
StickyAssignor.serializeTopicPartitionAssignment(new ConsumerUserData(assignment.get(consumer), Optional.of(assignor.generation())))));
|
|
||||||
}
|
|
||||||
subscriptions.remove(getConsumerName(5, 9));
|
|
||||||
|
|
||||||
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(assignor.isSticky());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testLargeAssignmentWithMultipleConsumersLeaving() {
|
|
||||||
Random rand = new Random();
|
|
||||||
int topicCount = 40;
|
|
||||||
int consumerCount = 200;
|
|
||||||
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
for (int i = 0; i < topicCount; i++)
|
|
||||||
partitionsPerTopic.put(getTopicName(i, topicCount), rand.nextInt(10) + 1);
|
|
||||||
|
|
||||||
for (int i = 0; i < consumerCount; i++) {
|
|
||||||
List<String> topics = new ArrayList<>();
|
|
||||||
for (int j = 0; j < rand.nextInt(20); j++)
|
|
||||||
topics.add(getTopicName(rand.nextInt(topicCount), topicCount));
|
|
||||||
subscriptions.put(getConsumerName(i, consumerCount), new Subscription(topics));
|
|
||||||
}
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
|
|
||||||
for (int i = 1; i < consumerCount; i++) {
|
|
||||||
String consumer = getConsumerName(i, consumerCount);
|
|
||||||
subscriptions.put(consumer,
|
|
||||||
new Subscription(subscriptions.get(consumer).topics(),
|
|
||||||
StickyAssignor.serializeTopicPartitionAssignment(new ConsumerUserData(assignment.get(consumer), Optional.of(assignor.generation())))));
|
|
||||||
}
|
|
||||||
for (int i = 0; i < 50; ++i) {
|
|
||||||
String c = getConsumerName(rand.nextInt(consumerCount), consumerCount);
|
|
||||||
subscriptions.remove(c);
|
|
||||||
}
|
|
||||||
|
|
||||||
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(assignor.isSticky());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testNewSubscription() {
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
for (int i = 1; i < 5; i++)
|
|
||||||
partitionsPerTopic.put(getTopicName(i, 5), 1);
|
|
||||||
|
|
||||||
for (int i = 0; i < 3; i++) {
|
|
||||||
List<String> topics = new ArrayList<>();
|
|
||||||
for (int j = i; j <= 3 * i - 2; j++)
|
|
||||||
topics.add(getTopicName(j, 5));
|
|
||||||
subscriptions.put(getConsumerName(i, 3), new Subscription(topics));
|
|
||||||
}
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
|
|
||||||
subscriptions.get(getConsumerName(0, 3)).topics().add(getTopicName(1, 5));
|
|
||||||
|
|
||||||
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(assignor.isSticky());
|
|
||||||
}
|
|
||||||
|
|
||||||
@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<String> topics = new ArrayList<>();
|
|
||||||
for (int i = 0; i < numTopics; ++i)
|
|
||||||
topics.add(getTopicName(i, maxNumTopics));
|
|
||||||
|
|
||||||
Map<String, Integer> 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);
|
|
||||||
|
|
||||||
for (int i = 0; i < numConsumers; ++i) {
|
|
||||||
List<String> sub = Utils.sorted(getRandomSublist(topics));
|
|
||||||
subscriptions.put(getConsumerName(i, maxNumConsumers), new Subscription(sub));
|
|
||||||
}
|
|
||||||
|
|
||||||
StickyAssignor assignor = new StickyAssignor();
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
|
|
||||||
subscriptions.clear();
|
|
||||||
for (int i = 0; i < numConsumers; ++i) {
|
|
||||||
List<String> sub = Utils.sorted(getRandomSublist(topics));
|
|
||||||
String consumer = getConsumerName(i, maxNumConsumers);
|
|
||||||
subscriptions.put(consumer,
|
|
||||||
new Subscription(sub, StickyAssignor.serializeTopicPartitionAssignment(new ConsumerUserData(assignment.get(consumer), Optional.of(assignor.generation())))));
|
|
||||||
}
|
|
||||||
|
|
||||||
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(assignor.isSticky());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testMoveExistingAssignments() {
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
for (int i = 1; i <= 6; i++)
|
|
||||||
partitionsPerTopic.put(String.format("topic%02d", i), 1);
|
|
||||||
|
|
||||||
subscriptions.put("consumer01",
|
|
||||||
new Subscription(topics("topic01", "topic02"),
|
|
||||||
StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(partitions(tp("topic01", 0)), Optional.of(assignor.generation())))));
|
|
||||||
subscriptions.put("consumer02",
|
|
||||||
new Subscription(topics("topic01", "topic02", "topic03", "topic04"),
|
|
||||||
StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(partitions(tp("topic02", 0), tp("topic03", 0)), Optional.of(assignor.generation())))));
|
|
||||||
subscriptions.put("consumer03",
|
|
||||||
new Subscription(topics("topic02", "topic03", "topic04", "topic05", "topic06"),
|
|
||||||
StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(partitions(tp("topic04", 0), tp("topic05", 0), tp("topic06", 0)), Optional.of(assignor.generation())))));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testStickiness() {
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
partitionsPerTopic.put("topic01", 3);
|
|
||||||
String consumer1 = "consumer01";
|
|
||||||
String consumer2 = "consumer02";
|
|
||||||
String consumer3 = "consumer03";
|
|
||||||
String consumer4 = "consumer04";
|
|
||||||
|
|
||||||
subscriptions.put(consumer1, new Subscription(topics("topic01")));
|
|
||||||
subscriptions.put(consumer2, new Subscription(topics("topic01")));
|
|
||||||
subscriptions.put(consumer3, new Subscription(topics("topic01")));
|
|
||||||
subscriptions.put(consumer4, new Subscription(topics("topic01")));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
Map<String, TopicPartition> partitionsAssigned = new HashMap<>();
|
|
||||||
|
|
||||||
Set<Entry<String, List<TopicPartition>>> assignments = assignment.entrySet();
|
|
||||||
for (Map.Entry<String, List<TopicPartition>> entry: assignments) {
|
|
||||||
String consumer = entry.getKey();
|
|
||||||
List<TopicPartition> 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(consumer1);
|
|
||||||
subscriptions.put(consumer2,
|
|
||||||
new Subscription(topics("topic01"),
|
|
||||||
StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(assignment.get("consumer02"), Optional.of(assignor.generation())))));
|
|
||||||
subscriptions.put(consumer3,
|
|
||||||
new Subscription(topics("topic01"),
|
|
||||||
StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(assignment.get("consumer03"), Optional.of(assignor.generation())))));
|
|
||||||
subscriptions.put(consumer4,
|
|
||||||
new Subscription(topics("topic01"),
|
|
||||||
StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(assignment.get("consumer04"), Optional.of(assignor.generation())))));
|
|
||||||
|
|
||||||
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(assignor.isSticky());
|
|
||||||
|
|
||||||
assignments = assignment.entrySet();
|
|
||||||
for (Map.Entry<String, List<TopicPartition>> entry: assignments) {
|
|
||||||
String consumer = entry.getKey();
|
|
||||||
List<TopicPartition> 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))));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testAssignmentUpdatedForDeletedTopic() {
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
partitionsPerTopic.put("topic01", 1);
|
|
||||||
partitionsPerTopic.put("topic03", 100);
|
|
||||||
subscriptions = Collections.singletonMap(consumerId, new Subscription(topics("topic01", "topic02", "topic03")));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
assertEquals(assignment.values().stream().mapToInt(topicPartitions -> topicPartitions.size()).sum(), 1 + 100);
|
|
||||||
assertEquals(Collections.singleton(consumerId), assignment.keySet());
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
}
|
|
||||||
|
|
||||||
@Test
|
|
||||||
public void testNoExceptionThrownWhenOnlySubscribedTopicDeleted() {
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
partitionsPerTopic.put(topic, 3);
|
|
||||||
subscriptions.put(consumerId, new Subscription(topics(topic)));
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
subscriptions.put(consumerId,
|
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(assignment.get(consumerId), Optional.of(1)))));
|
|
||||||
|
|
||||||
assignment = assignor.assign(Collections.emptyMap(), subscriptions);
|
|
||||||
assertEquals(assignment.size(), 1);
|
|
||||||
assertTrue(assignment.get(consumerId).isEmpty());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -662,40 +68,34 @@ public class StickyAssignorTest {
|
||||||
List<TopicPartition> r1partitions2 = assignment.get(consumer2);
|
List<TopicPartition> r1partitions2 = assignment.get(consumer2);
|
||||||
List<TopicPartition> r1partitions3 = assignment.get(consumer3);
|
List<TopicPartition> r1partitions3 = assignment.get(consumer3);
|
||||||
assertTrue(r1partitions1.size() == 2 && r1partitions2.size() == 2 && r1partitions3.size() == 2);
|
assertTrue(r1partitions1.size() == 2 && r1partitions2.size() == 2 && r1partitions3.size() == 2);
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
assertTrue(isFullyBalanced(assignment));
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
|
||||||
subscriptions.put(consumer1,
|
subscriptions.put(consumer1, buildSubscription(topics(topic), r1partitions1));
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
subscriptions.put(consumer2, buildSubscription(topics(topic), r1partitions2));
|
||||||
new ConsumerUserData(r1partitions1, Optional.of(1)))));
|
|
||||||
subscriptions.put(consumer2,
|
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(r1partitions2, Optional.of(1)))));
|
|
||||||
subscriptions.remove(consumer3);
|
subscriptions.remove(consumer3);
|
||||||
|
|
||||||
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
List<TopicPartition> r2partitions1 = assignment.get(consumer1);
|
List<TopicPartition> r2partitions1 = assignment.get(consumer1);
|
||||||
List<TopicPartition> r2partitions2 = assignment.get(consumer2);
|
List<TopicPartition> r2partitions2 = assignment.get(consumer2);
|
||||||
assertTrue(r2partitions1.size() == 3 && r2partitions2.size() == 3);
|
assertTrue(r2partitions1.size() == 3 && r2partitions2.size() == 3);
|
||||||
assertTrue(r2partitions1.containsAll(r1partitions1));
|
assertTrue(r2partitions1.containsAll(r1partitions1));
|
||||||
assertTrue(r2partitions2.containsAll(r1partitions2));
|
assertTrue(r2partitions2.containsAll(r1partitions2));
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
assertTrue(isFullyBalanced(assignment));
|
assertTrue(isFullyBalanced(assignment));
|
||||||
assertTrue(assignor.isSticky());
|
assertTrue(assignor.isSticky());
|
||||||
|
|
||||||
assertFalse(Collections.disjoint(r2partitions2, r1partitions3));
|
assertFalse(Collections.disjoint(r2partitions2, r1partitions3));
|
||||||
|
|
||||||
subscriptions.remove(consumer1);
|
subscriptions.remove(consumer1);
|
||||||
subscriptions.put(consumer2,
|
subscriptions.put(consumer2, buildSubscriptionWithGeneration(topics(topic), r2partitions2, 2));
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
subscriptions.put(consumer3, buildSubscriptionWithGeneration(topics(topic), r1partitions3, 1));
|
||||||
new ConsumerUserData(r2partitions2, Optional.of(2)))));
|
|
||||||
subscriptions.put(consumer3,
|
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(r1partitions3, Optional.of(1)))));
|
|
||||||
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
List<TopicPartition> r3partitions2 = assignment.get(consumer2);
|
List<TopicPartition> r3partitions2 = assignment.get(consumer2);
|
||||||
List<TopicPartition> r3partitions3 = assignment.get(consumer3);
|
List<TopicPartition> r3partitions3 = assignment.get(consumer3);
|
||||||
assertTrue(r3partitions2.size() == 3 && r3partitions3.size() == 3);
|
assertTrue(r3partitions2.size() == 3 && r3partitions3.size() == 3);
|
||||||
assertTrue(Collections.disjoint(r3partitions2, r3partitions3));
|
assertTrue(Collections.disjoint(r3partitions2, r3partitions3));
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
assertTrue(isFullyBalanced(assignment));
|
assertTrue(isFullyBalanced(assignment));
|
||||||
assertTrue(assignor.isSticky());
|
assertTrue(assignor.isSticky());
|
||||||
}
|
}
|
||||||
|
@ -717,31 +117,25 @@ public class StickyAssignorTest {
|
||||||
List<TopicPartition> r1partitions2 = assignment.get(consumer2);
|
List<TopicPartition> r1partitions2 = assignment.get(consumer2);
|
||||||
List<TopicPartition> r1partitions3 = assignment.get(consumer3);
|
List<TopicPartition> r1partitions3 = assignment.get(consumer3);
|
||||||
assertTrue(r1partitions1.size() == 2 && r1partitions2.size() == 2 && r1partitions3.size() == 2);
|
assertTrue(r1partitions1.size() == 2 && r1partitions2.size() == 2 && r1partitions3.size() == 2);
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
assertTrue(isFullyBalanced(assignment));
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
|
||||||
subscriptions.remove(consumer1);
|
subscriptions.remove(consumer1);
|
||||||
subscriptions.put(consumer2,
|
subscriptions.put(consumer2, buildSubscriptionWithGeneration(topics(topic), r1partitions2, 1));
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(r1partitions2, Optional.of(1)))));
|
|
||||||
subscriptions.remove(consumer3);
|
subscriptions.remove(consumer3);
|
||||||
|
|
||||||
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
List<TopicPartition> r2partitions2 = assignment.get(consumer2);
|
List<TopicPartition> r2partitions2 = assignment.get(consumer2);
|
||||||
assertEquals(6, r2partitions2.size());
|
assertEquals(6, r2partitions2.size());
|
||||||
assertTrue(r2partitions2.containsAll(r1partitions2));
|
assertTrue(r2partitions2.containsAll(r1partitions2));
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
assertTrue(isFullyBalanced(assignment));
|
assertTrue(isFullyBalanced(assignment));
|
||||||
assertTrue(assignor.isSticky());
|
assertTrue(assignor.isSticky());
|
||||||
|
|
||||||
subscriptions.put(consumer1,
|
subscriptions.put(consumer1, buildSubscriptionWithGeneration(topics(topic), r1partitions1, 1));
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
subscriptions.put(consumer2, buildSubscriptionWithGeneration(topics(topic), r2partitions2, 2));
|
||||||
new ConsumerUserData(r1partitions1, Optional.of(1)))));
|
subscriptions.put(consumer3, buildSubscriptionWithGeneration(topics(topic), r1partitions3, 1));
|
||||||
subscriptions.put(consumer2,
|
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(r2partitions2, Optional.of(2)))));
|
|
||||||
subscriptions.put(consumer3,
|
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(r1partitions3, Optional.of(1)))));
|
|
||||||
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
List<TopicPartition> r3partitions1 = assignment.get(consumer1);
|
List<TopicPartition> r3partitions1 = assignment.get(consumer1);
|
||||||
List<TopicPartition> r3partitions2 = assignment.get(consumer2);
|
List<TopicPartition> r3partitions2 = assignment.get(consumer2);
|
||||||
|
@ -750,7 +144,7 @@ public class StickyAssignorTest {
|
||||||
assertEquals(r1partitions1, r3partitions1);
|
assertEquals(r1partitions1, r3partitions1);
|
||||||
assertEquals(r1partitions2, r3partitions2);
|
assertEquals(r1partitions2, r3partitions2);
|
||||||
assertEquals(r1partitions3, r3partitions3);
|
assertEquals(r1partitions3, r3partitions3);
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
assertTrue(isFullyBalanced(assignment));
|
assertTrue(isFullyBalanced(assignment));
|
||||||
assertTrue(assignor.isSticky());
|
assertTrue(assignor.isSticky());
|
||||||
}
|
}
|
||||||
|
@ -777,15 +171,10 @@ public class StickyAssignorTest {
|
||||||
List<TopicPartition> c1partitions0 = partitions(tp0, tp1, tp4);
|
List<TopicPartition> c1partitions0 = partitions(tp0, tp1, tp4);
|
||||||
List<TopicPartition> c2partitions0 = partitions(tp0, tp2, tp3);
|
List<TopicPartition> c2partitions0 = partitions(tp0, tp2, tp3);
|
||||||
List<TopicPartition> c3partitions0 = partitions(tp3, tp4, tp5);
|
List<TopicPartition> c3partitions0 = partitions(tp3, tp4, tp5);
|
||||||
subscriptions.put(consumer1,
|
subscriptions.put(consumer1, buildSubscriptionWithGeneration(topics(topic), c1partitions0, 1));
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
subscriptions.put(consumer2, buildSubscriptionWithGeneration(topics(topic), c2partitions0, 1));
|
||||||
new ConsumerUserData(c1partitions0, Optional.of(1)))));
|
subscriptions.put(consumer3, buildSubscriptionWithGeneration(topics(topic), c3partitions0, 2));
|
||||||
subscriptions.put(consumer2,
|
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(c2partitions0, Optional.of(1)))));
|
|
||||||
subscriptions.put(consumer3,
|
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(c3partitions0, Optional.of(2)))));
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
List<TopicPartition> c1partitions = assignment.get(consumer1);
|
List<TopicPartition> c1partitions = assignment.get(consumer1);
|
||||||
List<TopicPartition> c2partitions = assignment.get(consumer2);
|
List<TopicPartition> c2partitions = assignment.get(consumer2);
|
||||||
|
@ -795,7 +184,7 @@ public class StickyAssignorTest {
|
||||||
assertTrue(c1partitions0.containsAll(c1partitions));
|
assertTrue(c1partitions0.containsAll(c1partitions));
|
||||||
assertTrue(c2partitions0.containsAll(c2partitions));
|
assertTrue(c2partitions0.containsAll(c2partitions));
|
||||||
assertTrue(c3partitions0.containsAll(c3partitions));
|
assertTrue(c3partitions0.containsAll(c3partitions));
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
assertTrue(isFullyBalanced(assignment));
|
assertTrue(isFullyBalanced(assignment));
|
||||||
assertTrue(assignor.isSticky());
|
assertTrue(assignor.isSticky());
|
||||||
}
|
}
|
||||||
|
@ -818,11 +207,8 @@ public class StickyAssignorTest {
|
||||||
|
|
||||||
List<TopicPartition> c1partitions0 = partitions(tp0, tp2);
|
List<TopicPartition> c1partitions0 = partitions(tp0, tp2);
|
||||||
List<TopicPartition> c2partitions0 = partitions(tp1);
|
List<TopicPartition> c2partitions0 = partitions(tp1);
|
||||||
subscriptions.put(consumer1,
|
subscriptions.put(consumer1, buildSubscriptionWithGeneration(topics(topic), c1partitions0, 1));
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
subscriptions.put(consumer2, buildSubscriptionWithOldSchema(topics(topic), c2partitions0));
|
||||||
new ConsumerUserData(c1partitions0, Optional.of(1)))));
|
|
||||||
subscriptions.put(consumer2,
|
|
||||||
new Subscription(topics(topic), serializeTopicPartitionAssignmentToOldSchema(c2partitions0)));
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
List<TopicPartition> c1partitions = assignment.get(consumer1);
|
List<TopicPartition> c1partitions = assignment.get(consumer1);
|
||||||
List<TopicPartition> c2partitions = assignment.get(consumer2);
|
List<TopicPartition> c2partitions = assignment.get(consumer2);
|
||||||
|
@ -831,45 +217,17 @@ public class StickyAssignorTest {
|
||||||
assertTrue(c1partitions.size() == 1 && c2partitions.size() == 1 && c3partitions.size() == 1);
|
assertTrue(c1partitions.size() == 1 && c2partitions.size() == 1 && c3partitions.size() == 1);
|
||||||
assertTrue(c1partitions0.containsAll(c1partitions));
|
assertTrue(c1partitions0.containsAll(c1partitions));
|
||||||
assertTrue(c2partitions0.containsAll(c2partitions));
|
assertTrue(c2partitions0.containsAll(c2partitions));
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
assertTrue(isFullyBalanced(assignment));
|
assertTrue(isFullyBalanced(assignment));
|
||||||
assertTrue(assignor.isSticky());
|
assertTrue(assignor.isSticky());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
private Subscription buildSubscriptionWithGeneration(List<String> topics, List<TopicPartition> partitions, int generation) {
|
||||||
public void testConflictingPreviousAssignments() {
|
return new Subscription(topics,
|
||||||
String consumer1 = "consumer1";
|
serializeTopicPartitionAssignment(new MemberData(partitions, Optional.of(generation))));
|
||||||
String consumer2 = "consumer2";
|
|
||||||
|
|
||||||
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
|
||||||
partitionsPerTopic.put(topic, 2);
|
|
||||||
subscriptions.put(consumer1, new Subscription(topics(topic)));
|
|
||||||
subscriptions.put(consumer2, new Subscription(topics(topic)));
|
|
||||||
|
|
||||||
TopicPartition tp0 = new TopicPartition(topic, 0);
|
|
||||||
TopicPartition tp1 = new TopicPartition(topic, 1);
|
|
||||||
|
|
||||||
// both c1 and c2 have partition 1 assigned to them in generation 1
|
|
||||||
List<TopicPartition> c1partitions0 = partitions(tp0, tp1);
|
|
||||||
List<TopicPartition> c2partitions0 = partitions(tp0, tp1);
|
|
||||||
subscriptions.put(consumer1,
|
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(c1partitions0, Optional.of(1)))));
|
|
||||||
subscriptions.put(consumer2,
|
|
||||||
new Subscription(topics(topic), StickyAssignor.serializeTopicPartitionAssignment(
|
|
||||||
new ConsumerUserData(c2partitions0, Optional.of(1)))));
|
|
||||||
|
|
||||||
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
|
||||||
List<TopicPartition> c1partitions = assignment.get(consumer1);
|
|
||||||
List<TopicPartition> c2partitions = assignment.get(consumer2);
|
|
||||||
|
|
||||||
assertTrue(c1partitions.size() == 1 && c2partitions.size() == 1);
|
|
||||||
verifyValidityAndBalance(subscriptions, assignment);
|
|
||||||
assertTrue(isFullyBalanced(assignment));
|
|
||||||
assertTrue(assignor.isSticky());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static ByteBuffer serializeTopicPartitionAssignmentToOldSchema(List<TopicPartition> partitions) {
|
private static Subscription buildSubscriptionWithOldSchema(List<String> topics, List<TopicPartition> partitions) {
|
||||||
Struct struct = new Struct(StickyAssignor.STICKY_ASSIGNOR_USER_DATA_V0);
|
Struct struct = new Struct(StickyAssignor.STICKY_ASSIGNOR_USER_DATA_V0);
|
||||||
List<Struct> topicAssignments = new ArrayList<>();
|
List<Struct> topicAssignments = new ArrayList<>();
|
||||||
for (Map.Entry<String, List<Integer>> topicEntry : CollectionUtils.groupPartitionsByTopic(partitions).entrySet()) {
|
for (Map.Entry<String, List<Integer>> topicEntry : CollectionUtils.groupPartitionsByTopic(partitions).entrySet()) {
|
||||||
|
@ -882,133 +240,7 @@ public class StickyAssignorTest {
|
||||||
ByteBuffer buffer = ByteBuffer.allocate(StickyAssignor.STICKY_ASSIGNOR_USER_DATA_V0.sizeOf(struct));
|
ByteBuffer buffer = ByteBuffer.allocate(StickyAssignor.STICKY_ASSIGNOR_USER_DATA_V0.sizeOf(struct));
|
||||||
StickyAssignor.STICKY_ASSIGNOR_USER_DATA_V0.write(buffer, struct);
|
StickyAssignor.STICKY_ASSIGNOR_USER_DATA_V0.write(buffer, struct);
|
||||||
buffer.flip();
|
buffer.flip();
|
||||||
return buffer;
|
|
||||||
}
|
|
||||||
|
|
||||||
private String getTopicName(int i, int maxNum) {
|
return new Subscription(topics, buffer);
|
||||||
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<String> topics(String... topics) {
|
|
||||||
return Arrays.asList(topics);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static List<TopicPartition> partitions(TopicPartition... partitions) {
|
|
||||||
return Arrays.asList(partitions);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static TopicPartition tp(String topic, int partition) {
|
|
||||||
return new TopicPartition(topic, partition);
|
|
||||||
}
|
|
||||||
|
|
||||||
private static boolean isFullyBalanced(Map<String, List<TopicPartition>> assignment) {
|
|
||||||
int min = Integer.MAX_VALUE;
|
|
||||||
int max = Integer.MIN_VALUE;
|
|
||||||
for (List<TopicPartition> topicPartitions: assignment.values()) {
|
|
||||||
int size = topicPartitions.size();
|
|
||||||
if (size < min)
|
|
||||||
min = size;
|
|
||||||
if (size > max)
|
|
||||||
max = size;
|
|
||||||
}
|
|
||||||
return max - min <= 1;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static List<String> getRandomSublist(ArrayList<String> list) {
|
|
||||||
List<String> 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;
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* 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 subscriptions: topic subscriptions of each consumer
|
|
||||||
* @param assignments: given assignment for balance check
|
|
||||||
*/
|
|
||||||
private static void verifyValidityAndBalance(Map<String, Subscription> subscriptions, Map<String, List<TopicPartition>> assignments) {
|
|
||||||
int size = subscriptions.size();
|
|
||||||
assert size == assignments.size();
|
|
||||||
|
|
||||||
List<String> consumers = Utils.sorted(assignments.keySet());
|
|
||||||
|
|
||||||
for (int i = 0; i < size; ++i) {
|
|
||||||
String consumer = consumers.get(i);
|
|
||||||
List<TopicPartition> 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).topics().contains(partition.topic()));
|
|
||||||
|
|
||||||
if (i == size - 1)
|
|
||||||
continue;
|
|
||||||
|
|
||||||
for (int j = i + 1; j < size; ++j) {
|
|
||||||
String otherConsumer = consumers.get(j);
|
|
||||||
List<TopicPartition> otherPartitions = assignments.get(otherConsumer);
|
|
||||||
|
|
||||||
Set<TopicPartition> 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<String, List<Integer>> map = CollectionUtils.groupPartitionsByTopic(partitions);
|
|
||||||
Map<String, List<Integer>> otherMap = CollectionUtils.groupPartitionsByTopic(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));
|
|
||||||
}
|
|
||||||
|
|
||||||
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));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -0,0 +1,799 @@
|
||||||
|
/*
|
||||||
|
* 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.internals;
|
||||||
|
|
||||||
|
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.Random;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
|
import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription;
|
||||||
|
import org.apache.kafka.common.TopicPartition;
|
||||||
|
import org.apache.kafka.common.utils.CollectionUtils;
|
||||||
|
import org.apache.kafka.common.utils.Utils;
|
||||||
|
import org.junit.Before;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import static org.junit.Assert.assertEquals;
|
||||||
|
import static org.junit.Assert.assertFalse;
|
||||||
|
import static org.junit.Assert.assertTrue;
|
||||||
|
|
||||||
|
public abstract class AbstractStickyAssignorTest {
|
||||||
|
|
||||||
|
protected AbstractStickyAssignor assignor;
|
||||||
|
protected String consumerId = "consumer";
|
||||||
|
protected Map<String, Subscription> subscriptions;
|
||||||
|
protected String topic = "topic";
|
||||||
|
|
||||||
|
protected abstract AbstractStickyAssignor createAssignor();
|
||||||
|
|
||||||
|
protected abstract Subscription buildSubscription(List<String> topics, List<TopicPartition> partitions);
|
||||||
|
|
||||||
|
@Before
|
||||||
|
public void setUp() {
|
||||||
|
assignor = createAssignor();
|
||||||
|
|
||||||
|
if (subscriptions != null) {
|
||||||
|
subscriptions.clear();
|
||||||
|
} else {
|
||||||
|
subscriptions = new HashMap<>();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testOneConsumerNoTopic() {
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
subscriptions = Collections.singletonMap(consumerId, new Subscription(Collections.emptyList()));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
assertEquals(Collections.singleton(consumerId), assignment.keySet());
|
||||||
|
assertTrue(assignment.get(consumerId).isEmpty());
|
||||||
|
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testOneConsumerNonexistentTopic() {
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
partitionsPerTopic.put(topic, 0);
|
||||||
|
subscriptions = Collections.singletonMap(consumerId, new Subscription(topics(topic)));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
|
||||||
|
assertEquals(Collections.singleton(consumerId), assignment.keySet());
|
||||||
|
assertTrue(assignment.get(consumerId).isEmpty());
|
||||||
|
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testOneConsumerOneTopic() {
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
partitionsPerTopic.put(topic, 3);
|
||||||
|
subscriptions = Collections.singletonMap(consumerId, new Subscription(topics(topic)));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
assertEquals(partitions(tp(topic, 0), tp(topic, 1), tp(topic, 2)), assignment.get(consumerId));
|
||||||
|
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testOnlyAssignsPartitionsFromSubscribedTopics() {
|
||||||
|
String otherTopic = "other";
|
||||||
|
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
partitionsPerTopic.put(topic, 3);
|
||||||
|
partitionsPerTopic.put(otherTopic, 3);
|
||||||
|
subscriptions = Collections.singletonMap(consumerId, new Subscription(topics(topic)));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
assertEquals(partitions(tp(topic, 0), tp(topic, 1), tp(topic, 2)), assignment.get(consumerId));
|
||||||
|
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testOneConsumerMultipleTopics() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
|
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
partitionsPerTopic.put(topic1, 1);
|
||||||
|
partitionsPerTopic.put(topic2, 2);
|
||||||
|
subscriptions = Collections.singletonMap(consumerId, new Subscription(topics(topic1, topic2)));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
assertEquals(partitions(tp(topic1, 0), tp(topic2, 0), tp(topic2, 1)), assignment.get(consumerId));
|
||||||
|
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTwoConsumersOneTopicOnePartition() {
|
||||||
|
String consumer1 = "consumer1";
|
||||||
|
String consumer2 = "consumer2";
|
||||||
|
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
partitionsPerTopic.put(topic, 1);
|
||||||
|
|
||||||
|
subscriptions.put(consumer1, new Subscription(topics(topic)));
|
||||||
|
subscriptions.put(consumer2, new Subscription(topics(topic)));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
assertEquals(partitions(tp(topic, 0)), assignment.get(consumer1));
|
||||||
|
assertEquals(Collections.<TopicPartition>emptyList(), assignment.get(consumer2));
|
||||||
|
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTwoConsumersOneTopicTwoPartitions() {
|
||||||
|
String consumer1 = "consumer1";
|
||||||
|
String consumer2 = "consumer2";
|
||||||
|
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
partitionsPerTopic.put(topic, 2);
|
||||||
|
|
||||||
|
subscriptions.put(consumer1, new Subscription(topics(topic)));
|
||||||
|
subscriptions.put(consumer2, new Subscription(topics(topic)));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
assertEquals(partitions(tp(topic, 0)), assignment.get(consumer1));
|
||||||
|
assertEquals(partitions(tp(topic, 1)), assignment.get(consumer2));
|
||||||
|
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMultipleConsumersMixedTopicSubscriptions() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
|
String consumer1 = "consumer1";
|
||||||
|
String consumer2 = "consumer2";
|
||||||
|
String consumer3 = "consumer3";
|
||||||
|
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
partitionsPerTopic.put(topic1, 3);
|
||||||
|
partitionsPerTopic.put(topic2, 2);
|
||||||
|
|
||||||
|
subscriptions.put(consumer1, new Subscription(topics(topic1)));
|
||||||
|
subscriptions.put(consumer2, new Subscription(topics(topic1, topic2)));
|
||||||
|
subscriptions.put(consumer3, new Subscription(topics(topic1)));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
assertEquals(partitions(tp(topic1, 0), tp(topic1, 2)), assignment.get(consumer1));
|
||||||
|
assertEquals(partitions(tp(topic2, 0), tp(topic2, 1)), assignment.get(consumer2));
|
||||||
|
assertEquals(partitions(tp(topic1, 1)), assignment.get(consumer3));
|
||||||
|
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testTwoConsumersTwoTopicsSixPartitions() {
|
||||||
|
String topic1 = "topic1";
|
||||||
|
String topic2 = "topic2";
|
||||||
|
String consumer1 = "consumer1";
|
||||||
|
String consumer2 = "consumer2";
|
||||||
|
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
partitionsPerTopic.put(topic1, 3);
|
||||||
|
partitionsPerTopic.put(topic2, 3);
|
||||||
|
|
||||||
|
subscriptions.put(consumer1, new Subscription(topics(topic1, topic2)));
|
||||||
|
subscriptions.put(consumer2, new Subscription(topics(topic1, topic2)));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
assertEquals(partitions(tp(topic1, 0), tp(topic1, 2), tp(topic2, 1)), assignment.get(consumer1));
|
||||||
|
assertEquals(partitions(tp(topic1, 1), tp(topic2, 0), tp(topic2, 2)), assignment.get(consumer2));
|
||||||
|
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAddRemoveConsumerOneTopic() {
|
||||||
|
String consumer1 = "consumer1";
|
||||||
|
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
partitionsPerTopic.put(topic, 3);
|
||||||
|
subscriptions.put(consumer1, new Subscription(topics(topic)));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
assertEquals(partitions(tp(topic, 0), tp(topic, 1), tp(topic, 2)), assignment.get(consumer1));
|
||||||
|
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
|
||||||
|
String consumer2 = "consumer2";
|
||||||
|
subscriptions.put(consumer1, buildSubscription(topics(topic), assignment.get(consumer1)));
|
||||||
|
subscriptions.put(consumer2, buildSubscription(topics(topic), Collections.emptyList()));
|
||||||
|
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertEquals(partitions(tp(topic, 2), tp(topic, 1)), assignment.get(consumer1));
|
||||||
|
assertEquals(partitions(tp(topic, 0)), assignment.get(consumer2));
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
assertTrue(assignor.isSticky());
|
||||||
|
|
||||||
|
subscriptions.remove(consumer1);
|
||||||
|
subscriptions.put(consumer2, buildSubscription(topics(topic), assignment.get(consumer2)));
|
||||||
|
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
assertEquals(new HashSet<>(partitions(tp(topic, 2), tp(topic, 1), tp(topic, 0))),
|
||||||
|
new HashSet<>(assignment.get(consumer2)));
|
||||||
|
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
assertTrue(assignor.isSticky());
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* 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<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
for (int i = 1; i <= 5; i++)
|
||||||
|
partitionsPerTopic.put(String.format("topic%d", i), (i % 2) + 1);
|
||||||
|
|
||||||
|
subscriptions.put("consumer1",
|
||||||
|
new Subscription(topics("topic1", "topic2", "topic3", "topic4", "topic5")));
|
||||||
|
subscriptions.put("consumer2",
|
||||||
|
new Subscription(topics("topic1", "topic3", "topic5")));
|
||||||
|
subscriptions.put("consumer3",
|
||||||
|
new Subscription(topics("topic1", "topic3", "topic5")));
|
||||||
|
subscriptions.put("consumer4",
|
||||||
|
new Subscription(topics("topic1", "topic2", "topic3", "topic4", "topic5")));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAddRemoveTopicTwoConsumers() {
|
||||||
|
String consumer1 = "consumer";
|
||||||
|
String consumer2 = "consumer2";
|
||||||
|
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
partitionsPerTopic.put(topic, 3);
|
||||||
|
subscriptions.put(consumer1, new Subscription(topics(topic)));
|
||||||
|
subscriptions.put(consumer2, new Subscription(topics(topic)));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
// verify balance
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
// verify stickiness
|
||||||
|
List<TopicPartition> consumer1Assignment1 = assignment.get(consumer1);
|
||||||
|
List<TopicPartition> 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, buildSubscription(topics(topic, topic2), assignment.get(consumer1)));
|
||||||
|
subscriptions.put(consumer2, buildSubscription(topics(topic, topic2), assignment.get(consumer2)));
|
||||||
|
|
||||||
|
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
// verify balance
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
// verify stickiness
|
||||||
|
List<TopicPartition> consumer1assignment = assignment.get(consumer1);
|
||||||
|
List<TopicPartition> consumer2assignment = assignment.get(consumer2);
|
||||||
|
assertTrue(consumer1assignment.size() == 3 && consumer2assignment.size() == 3);
|
||||||
|
assertTrue(consumer1assignment.containsAll(consumer1Assignment1));
|
||||||
|
assertTrue(consumer2assignment.containsAll(consumer2Assignment1));
|
||||||
|
assertTrue(assignor.isSticky());
|
||||||
|
|
||||||
|
partitionsPerTopic.remove(topic);
|
||||||
|
subscriptions.put(consumer1, buildSubscription(topics(topic2), assignment.get(consumer1)));
|
||||||
|
subscriptions.put(consumer2, buildSubscription(topics(topic2), assignment.get(consumer2)));
|
||||||
|
|
||||||
|
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
// verify balance
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
// verify stickiness
|
||||||
|
List<TopicPartition> consumer1Assignment3 = assignment.get(consumer1);
|
||||||
|
List<TopicPartition> consumer2Assignment3 = assignment.get(consumer2);
|
||||||
|
assertTrue((consumer1Assignment3.size() == 1 && consumer2Assignment3.size() == 2) ||
|
||||||
|
(consumer1Assignment3.size() == 2 && consumer2Assignment3.size() == 1));
|
||||||
|
assertTrue(consumer1assignment.containsAll(consumer1Assignment3));
|
||||||
|
assertTrue(consumer2assignment.containsAll(consumer2Assignment3));
|
||||||
|
assertTrue(assignor.isSticky());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testReassignmentAfterOneConsumerLeaves() {
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
for (int i = 1; i < 20; i++)
|
||||||
|
partitionsPerTopic.put(getTopicName(i, 20), i);
|
||||||
|
|
||||||
|
for (int i = 1; i < 20; i++) {
|
||||||
|
List<String> topics = new ArrayList<>();
|
||||||
|
for (int j = 1; j <= i; j++)
|
||||||
|
topics.add(getTopicName(j, 20));
|
||||||
|
subscriptions.put(getConsumerName(i, 20), new Subscription(topics));
|
||||||
|
}
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
|
||||||
|
for (int i = 1; i < 20; i++) {
|
||||||
|
String consumer = getConsumerName(i, 20);
|
||||||
|
subscriptions.put(consumer,
|
||||||
|
buildSubscription(subscriptions.get(consumer).topics(), assignment.get(consumer)));
|
||||||
|
}
|
||||||
|
subscriptions.remove("consumer10");
|
||||||
|
|
||||||
|
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(assignor.isSticky());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testReassignmentAfterOneConsumerAdded() {
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
partitionsPerTopic.put("topic", 20);
|
||||||
|
|
||||||
|
for (int i = 1; i < 10; i++)
|
||||||
|
subscriptions.put(getConsumerName(i, 10),
|
||||||
|
new Subscription(topics("topic")));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
|
||||||
|
// add a new consumer
|
||||||
|
subscriptions.put(getConsumerName(10, 10), new Subscription(topics("topic")));
|
||||||
|
|
||||||
|
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(assignor.isSticky());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testSameSubscriptions() {
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
for (int i = 1; i < 15; i++)
|
||||||
|
partitionsPerTopic.put(getTopicName(i, 15), i);
|
||||||
|
|
||||||
|
for (int i = 1; i < 9; i++) {
|
||||||
|
List<String> topics = new ArrayList<>();
|
||||||
|
for (int j = 1; j <= partitionsPerTopic.size(); j++)
|
||||||
|
topics.add(getTopicName(j, 15));
|
||||||
|
subscriptions.put(getConsumerName(i, 9), new Subscription(topics));
|
||||||
|
}
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
|
||||||
|
for (int i = 1; i < 9; i++) {
|
||||||
|
String consumer = getConsumerName(i, 9);
|
||||||
|
subscriptions.put(consumer,
|
||||||
|
buildSubscription(subscriptions.get(consumer).topics(), assignment.get(consumer)));
|
||||||
|
}
|
||||||
|
subscriptions.remove(getConsumerName(5, 9));
|
||||||
|
|
||||||
|
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(assignor.isSticky());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testLargeAssignmentWithMultipleConsumersLeaving() {
|
||||||
|
Random rand = new Random();
|
||||||
|
int topicCount = 40;
|
||||||
|
int consumerCount = 200;
|
||||||
|
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
for (int i = 0; i < topicCount; i++)
|
||||||
|
partitionsPerTopic.put(getTopicName(i, topicCount), rand.nextInt(10) + 1);
|
||||||
|
|
||||||
|
for (int i = 0; i < consumerCount; i++) {
|
||||||
|
List<String> topics = new ArrayList<>();
|
||||||
|
for (int j = 0; j < rand.nextInt(20); j++)
|
||||||
|
topics.add(getTopicName(rand.nextInt(topicCount), topicCount));
|
||||||
|
subscriptions.put(getConsumerName(i, consumerCount), new Subscription(topics));
|
||||||
|
}
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
|
||||||
|
for (int i = 1; i < consumerCount; i++) {
|
||||||
|
String consumer = getConsumerName(i, consumerCount);
|
||||||
|
subscriptions.put(consumer,
|
||||||
|
buildSubscription(subscriptions.get(consumer).topics(), assignment.get(consumer)));
|
||||||
|
}
|
||||||
|
for (int i = 0; i < 50; ++i) {
|
||||||
|
String c = getConsumerName(rand.nextInt(consumerCount), consumerCount);
|
||||||
|
subscriptions.remove(c);
|
||||||
|
}
|
||||||
|
|
||||||
|
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(assignor.isSticky());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testNewSubscription() {
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
for (int i = 1; i < 5; i++)
|
||||||
|
partitionsPerTopic.put(getTopicName(i, 5), 1);
|
||||||
|
|
||||||
|
for (int i = 0; i < 3; i++) {
|
||||||
|
List<String> topics = new ArrayList<>();
|
||||||
|
for (int j = i; j <= 3 * i - 2; j++)
|
||||||
|
topics.add(getTopicName(j, 5));
|
||||||
|
subscriptions.put(getConsumerName(i, 3), new Subscription(topics));
|
||||||
|
}
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
|
||||||
|
subscriptions.get(getConsumerName(0, 3)).topics().add(getTopicName(1, 5));
|
||||||
|
|
||||||
|
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(assignor.isSticky());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testMoveExistingAssignments() {
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
for (int i = 1; i <= 6; i++)
|
||||||
|
partitionsPerTopic.put(String.format("topic%02d", i), 1);
|
||||||
|
|
||||||
|
subscriptions.put("consumer01",
|
||||||
|
buildSubscription(topics("topic01", "topic02"),
|
||||||
|
partitions(tp("topic01", 0))));
|
||||||
|
subscriptions.put("consumer02",
|
||||||
|
buildSubscription(topics("topic01", "topic02", "topic03", "topic04"),
|
||||||
|
partitions(tp("topic02", 0), tp("topic03", 0))));
|
||||||
|
subscriptions.put("consumer03",
|
||||||
|
buildSubscription(topics("topic02", "topic03", "topic04", "topic05", "topic06"),
|
||||||
|
partitions(tp("topic04", 0), tp("topic05", 0), tp("topic06", 0))));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testStickiness() {
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
partitionsPerTopic.put("topic01", 3);
|
||||||
|
String consumer1 = "consumer01";
|
||||||
|
String consumer2 = "consumer02";
|
||||||
|
String consumer3 = "consumer03";
|
||||||
|
String consumer4 = "consumer04";
|
||||||
|
|
||||||
|
subscriptions.put(consumer1, new Subscription(topics("topic01")));
|
||||||
|
subscriptions.put(consumer2, new Subscription(topics("topic01")));
|
||||||
|
subscriptions.put(consumer3, new Subscription(topics("topic01")));
|
||||||
|
subscriptions.put(consumer4, new Subscription(topics("topic01")));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
Map<String, TopicPartition> partitionsAssigned = new HashMap<>();
|
||||||
|
|
||||||
|
Set<Map.Entry<String, List<TopicPartition>>> assignments = assignment.entrySet();
|
||||||
|
for (Map.Entry<String, List<TopicPartition>> entry: assignments) {
|
||||||
|
String consumer = entry.getKey();
|
||||||
|
List<TopicPartition> 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(consumer1);
|
||||||
|
subscriptions.put(consumer2,
|
||||||
|
buildSubscription(topics("topic01"), assignment.get(consumer2)));
|
||||||
|
subscriptions.put(consumer3,
|
||||||
|
buildSubscription(topics("topic01"), assignment.get(consumer3)));
|
||||||
|
subscriptions.put(consumer4,
|
||||||
|
buildSubscription(topics("topic01"), assignment.get(consumer4)));
|
||||||
|
|
||||||
|
|
||||||
|
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(assignor.isSticky());
|
||||||
|
|
||||||
|
assignments = assignment.entrySet();
|
||||||
|
for (Map.Entry<String, List<TopicPartition>> entry: assignments) {
|
||||||
|
String consumer = entry.getKey();
|
||||||
|
List<TopicPartition> 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))));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testAssignmentUpdatedForDeletedTopic() {
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
partitionsPerTopic.put("topic01", 1);
|
||||||
|
partitionsPerTopic.put("topic03", 100);
|
||||||
|
subscriptions = Collections.singletonMap(consumerId, new Subscription(topics("topic01", "topic02", "topic03")));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
assertEquals(assignment.values().stream().mapToInt(topicPartitions -> topicPartitions.size()).sum(), 1 + 100);
|
||||||
|
assertEquals(Collections.singleton(consumerId), assignment.keySet());
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testNoExceptionThrownWhenOnlySubscribedTopicDeleted() {
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
partitionsPerTopic.put(topic, 3);
|
||||||
|
subscriptions.put(consumerId, new Subscription(topics(topic)));
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
subscriptions.put(consumerId, buildSubscription(topics(topic), assignment.get(consumerId)));
|
||||||
|
|
||||||
|
assignment = assignor.assign(Collections.emptyMap(), subscriptions);
|
||||||
|
assertEquals(assignment.size(), 1);
|
||||||
|
assertTrue(assignment.get(consumerId).isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testConflictingPreviousAssignments() {
|
||||||
|
String consumer1 = "consumer1";
|
||||||
|
String consumer2 = "consumer2";
|
||||||
|
|
||||||
|
Map<String, Integer> partitionsPerTopic = new HashMap<>();
|
||||||
|
partitionsPerTopic.put(topic, 2);
|
||||||
|
subscriptions.put(consumer1, new Subscription(topics(topic)));
|
||||||
|
subscriptions.put(consumer2, new Subscription(topics(topic)));
|
||||||
|
|
||||||
|
TopicPartition tp0 = new TopicPartition(topic, 0);
|
||||||
|
TopicPartition tp1 = new TopicPartition(topic, 1);
|
||||||
|
|
||||||
|
// both c1 and c2 have partition 1 assigned to them in generation 1
|
||||||
|
List<TopicPartition> c1partitions0 = partitions(tp0, tp1);
|
||||||
|
List<TopicPartition> c2partitions0 = partitions(tp0, tp1);
|
||||||
|
subscriptions.put(consumer1, buildSubscription(topics(topic), c1partitions0));
|
||||||
|
subscriptions.put(consumer2, buildSubscription(topics(topic), c2partitions0));
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
List<TopicPartition> c1partitions = assignment.get(consumer1);
|
||||||
|
List<TopicPartition> c2partitions = assignment.get(consumer2);
|
||||||
|
|
||||||
|
assertTrue(c1partitions.size() == 1 && c2partitions.size() == 1);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(isFullyBalanced(assignment));
|
||||||
|
assertTrue(assignor.isSticky());
|
||||||
|
}
|
||||||
|
|
||||||
|
@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<String> topics = new ArrayList<>();
|
||||||
|
for (int i = 0; i < numTopics; ++i)
|
||||||
|
topics.add(getTopicName(i, maxNumTopics));
|
||||||
|
|
||||||
|
Map<String, Integer> 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);
|
||||||
|
|
||||||
|
for (int i = 0; i < numConsumers; ++i) {
|
||||||
|
List<String> sub = Utils.sorted(getRandomSublist(topics));
|
||||||
|
subscriptions.put(getConsumerName(i, maxNumConsumers), new Subscription(sub));
|
||||||
|
}
|
||||||
|
|
||||||
|
assignor = createAssignor();
|
||||||
|
|
||||||
|
Map<String, List<TopicPartition>> assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
|
||||||
|
subscriptions.clear();
|
||||||
|
for (int i = 0; i < numConsumers; ++i) {
|
||||||
|
List<String> sub = Utils.sorted(getRandomSublist(topics));
|
||||||
|
String consumer = getConsumerName(i, maxNumConsumers);
|
||||||
|
subscriptions.put(consumer, buildSubscription(sub, assignment.get(consumer)));
|
||||||
|
}
|
||||||
|
|
||||||
|
assignment = assignor.assign(partitionsPerTopic, subscriptions);
|
||||||
|
verifyValidityAndBalance(subscriptions, assignment, partitionsPerTopic);
|
||||||
|
assertTrue(assignor.isSticky());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
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();
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static List<String> topics(String... topics) {
|
||||||
|
return Arrays.asList(topics);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static List<TopicPartition> partitions(TopicPartition... partitions) {
|
||||||
|
return Arrays.asList(partitions);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static TopicPartition tp(String topic, int partition) {
|
||||||
|
return new TopicPartition(topic, partition);
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static boolean isFullyBalanced(Map<String, List<TopicPartition>> assignment) {
|
||||||
|
int min = Integer.MAX_VALUE;
|
||||||
|
int max = Integer.MIN_VALUE;
|
||||||
|
for (List<TopicPartition> topicPartitions: assignment.values()) {
|
||||||
|
int size = topicPartitions.size();
|
||||||
|
if (size < min)
|
||||||
|
min = size;
|
||||||
|
if (size > max)
|
||||||
|
max = size;
|
||||||
|
}
|
||||||
|
return max - min <= 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected static List<String> getRandomSublist(ArrayList<String> list) {
|
||||||
|
List<String> 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;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Verifies that the given assignment is valid 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 subscriptions: topic subscriptions of each consumer
|
||||||
|
* @param assignments: given assignment for balance check
|
||||||
|
* @param partitionsPerTopic: number of partitions per topic
|
||||||
|
*/
|
||||||
|
protected void verifyValidityAndBalance(Map<String, Subscription> subscriptions,
|
||||||
|
Map<String, List<TopicPartition>> assignments,
|
||||||
|
Map<String, Integer> partitionsPerTopic) {
|
||||||
|
int size = subscriptions.size();
|
||||||
|
assert size == assignments.size();
|
||||||
|
|
||||||
|
List<String> consumers = Utils.sorted(assignments.keySet());
|
||||||
|
|
||||||
|
for (int i = 0; i < size; ++i) {
|
||||||
|
String consumer = consumers.get(i);
|
||||||
|
List<TopicPartition> 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).topics().contains(partition.topic()));
|
||||||
|
|
||||||
|
if (i == size - 1)
|
||||||
|
continue;
|
||||||
|
|
||||||
|
for (int j = i + 1; j < size; ++j) {
|
||||||
|
String otherConsumer = consumers.get(j);
|
||||||
|
List<TopicPartition> otherPartitions = assignments.get(otherConsumer);
|
||||||
|
|
||||||
|
Set<TopicPartition> 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<String, List<Integer>> map = CollectionUtils.groupPartitionsByTopic(partitions);
|
||||||
|
Map<String, List<Integer>> otherMap = CollectionUtils.groupPartitionsByTopic(otherPartitions);
|
||||||
|
|
||||||
|
if (len > otherLen) {
|
||||||
|
for (String topic: map.keySet())
|
||||||
|
if (otherMap.containsKey(topic))
|
||||||
|
//assertTrue(true);
|
||||||
|
assertFalse("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));
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
if (otherLen > len) {
|
||||||
|
for (String topic: otherMap.keySet())
|
||||||
|
if (otherMap.containsKey(topic))
|
||||||
|
//assertTrue(true);
|
||||||
|
assertFalse("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));
|
||||||
|
|
||||||
|
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -30,12 +30,14 @@
|
||||||
avoid potential misuse. The constructor <code>TopicAuthorizationException(String)</code> which was previously used for a single
|
avoid potential misuse. The constructor <code>TopicAuthorizationException(String)</code> which was previously used for a single
|
||||||
unauthorized topic was changed similarly.
|
unauthorized topic was changed similarly.
|
||||||
</li>
|
</li>
|
||||||
<li>The internal <code>PartitionAssignor</code> interface has been deprecated and replaced with a new <code>ConsumerPartitionAssignor</code> in the public API. Users
|
<li>The internal <code>PartitionAssignor</code> interface has been deprecated and replaced with a new <code>ConsumerPartitionAssignor</code> in the public API. Some
|
||||||
implementing a custom PartitionAssignor should migrate to the new interface as soon as possible.</li>
|
methods/signatures are slightly different between the two interfaces. Users implementing a custom PartitionAssignor should migrate to the new interface as soon as possible.
|
||||||
|
</li>
|
||||||
<li>The <code>DefaultPartitioner</code> now uses a sticky partitioning strategy. This means that records for specific topic with null keys and no assigned partition
|
<li>The <code>DefaultPartitioner</code> now uses a sticky partitioning strategy. This means that records for specific topic with null keys and no assigned partition
|
||||||
will be sent to the same partition until the batch is ready to be sent. When a new batch is created, a new partition is chosen. This decreases latency to produce, but
|
will be sent to the same partition until the batch is ready to be sent. When a new batch is created, a new partition is chosen. This decreases latency to produce, but
|
||||||
it may result in uneven distribution of records across partitions in edge cases. Generally users will not be impacted, but this difference may be noticeable in tests and
|
it may result in uneven distribution of records across partitions in edge cases. Generally users will not be impacted, but this difference may be noticeable in tests and
|
||||||
other situations producing records for a very short amount of time.</li>
|
other situations producing records for a very short amount of time.
|
||||||
|
</li>
|
||||||
</ul>
|
</ul>
|
||||||
|
|
||||||
<h4><a id="upgrade_2_3_0" href="#upgrade_2_3_0">Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x, 0.11.0.x, 1.0.x, 1.1.x, 2.0.x or 2.1.x or 2.2.x to 2.3.0</a></h4>
|
<h4><a id="upgrade_2_3_0" href="#upgrade_2_3_0">Upgrading from 0.8.x, 0.9.x, 0.10.0.x, 0.10.1.x, 0.10.2.x, 0.11.0.x, 1.0.x, 1.1.x, 2.0.x or 2.1.x or 2.2.x to 2.3.0</a></h4>
|
||||||
|
|
|
@ -334,7 +334,7 @@ public class EosIntegrationTest {
|
||||||
|
|
||||||
TestUtils.waitForCondition(
|
TestUtils.waitForCondition(
|
||||||
() -> commitRequested.get() == 2, MAX_WAIT_TIME_MS,
|
() -> commitRequested.get() == 2, MAX_WAIT_TIME_MS,
|
||||||
"SteamsTasks did not request commit.");
|
"StreamsTasks did not request commit.");
|
||||||
|
|
||||||
writeInputData(uncommittedDataBeforeFailure);
|
writeInputData(uncommittedDataBeforeFailure);
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue