Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-4402: make the KafkaProducer true round robin per topic #2128

Closed
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicInteger;

import org.apache.kafka.clients.producer.Partitioner;
Expand All @@ -35,7 +37,7 @@
*/
public class DefaultPartitioner implements Partitioner {

private final AtomicInteger counter = new AtomicInteger(new Random().nextInt());
private final ConcurrentMap<String, AtomicInteger> topicCounterMap = new ConcurrentHashMap<>();

public void configure(Map<String, ?> configs) {}

Expand All @@ -53,7 +55,7 @@ public int partition(String topic, Object key, byte[] keyBytes, Object value, by
List<PartitionInfo> partitions = cluster.partitionsForTopic(topic);
int numPartitions = partitions.size();
if (keyBytes == null) {
int nextValue = counter.getAndIncrement();
int nextValue = nextValue(topic);
List<PartitionInfo> availablePartitions = cluster.availablePartitionsForTopic(topic);
if (availablePartitions.size() > 0) {
int part = Utils.toPositive(nextValue) % availablePartitions.size();
Expand All @@ -68,6 +70,18 @@ public int partition(String topic, Object key, byte[] keyBytes, Object value, by
}
}

private int nextValue(String topic) {
AtomicInteger counter = topicCounterMap.get(topic);
if (null == counter) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This isn't strictly thread safe since you might overwrite values in the map. It won't cause a crash, but not perfectly consistent behavior. Was this intentional since it only matters for concurrent access the first time a topic is accessed? Or should this entire method be synchronized to guarantee the behavior you're expecting (in which case the Map itself doesn't need to be a concurrent implementation)?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yeah, I was aware of it and chose to do it in this way.
As this method is a hot method which may be called by every producer.send() if no key is provided, put it as 'synchronized' may potentially impact the performance among the threads.

while the null check may be both true for the initial concurrent threads, as you said, it only impact that very first few calls. further calls are not impacted.

counter = new AtomicInteger(new Random().nextInt());
AtomicInteger currentCounter = topicCounterMap.putIfAbsent(topic, counter);

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

looks like, there is a race condition putting the counter to the map

if (currentCounter != null) {
counter = currentCounter;
}
}
return counter.getAndIncrement();
}

public void close() {}

}
Original file line number Diff line number Diff line change
Expand Up @@ -16,11 +16,13 @@
import org.apache.kafka.common.Cluster;
import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
import org.junit.Assert;
import org.junit.Test;

import java.util.Collections;
import java.util.HashMap;
import java.util.List;

import java.util.Map;
import static java.util.Arrays.asList;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
Expand Down Expand Up @@ -62,4 +64,35 @@ public void testRoundRobinWithUnavailablePartitions() {
}
assertEquals("The distribution between two available partitions should be even", countForPart0, countForPart2);
}

@Test
public void testRoundRobin() throws InterruptedException {
final String topicA = "topicA";
final String topicB = "topicB";

List<PartitionInfo> allPartitions = asList(new PartitionInfo(topicA, 0, node0, nodes, nodes),
new PartitionInfo(topicA, 1, node1, nodes, nodes),
new PartitionInfo(topicA, 2, node2, nodes, nodes),
new PartitionInfo(topicB, 0, node0, nodes, nodes)
);
Cluster testCluster = new Cluster("clusterId", asList(node0, node1, node2), allPartitions,
Collections.<String>emptySet(), Collections.<String>emptySet());

final Map<Integer, Integer> partitionCount = new HashMap<>();

for (int i = 0; i < 30; ++i) {
int partition = partitioner.partition(topicA, null, null, null, null, testCluster);
Integer count = partitionCount.get(partition);
if (null == count) count = 0;
partitionCount.put(partition, count + 1);

if (i % 5 == 0) {
partitioner.partition(topicB, null, null, null, null, testCluster);
}
}

Assert.assertEquals(10, (int) partitionCount.get(0));
Assert.assertEquals(10, (int) partitionCount.get(1));
Assert.assertEquals(10, (int) partitionCount.get(2));
}
}