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
Expand Up @@ -19,6 +19,7 @@
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;

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

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

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

Expand All @@ -53,7 +54,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 = getNextValue(topic);
List<PartitionInfo> availablePartitions = cluster.availablePartitionsForTopic(topic);
if (availablePartitions.size() > 0) {
int part = Utils.toPositive(nextValue) % availablePartitions.size();
Expand All @@ -68,6 +69,15 @@ public int partition(String topic, Object key, byte[] keyBytes, Object value, by
}
}

private int getNextValue(String topic) {
Copy link
Contributor

Choose a reason for hiding this comment

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

In Kafka, we generally avoid the get prefix. So nextValue would probably be better.

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, updated.

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());
topicCounterMap.put(topic, counter);
Copy link

Choose a reason for hiding this comment

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

getNextValue is not synchronised. There is a race in this line. Should be replaced with currentCounter = topicCounterMap.putIfAbsent(topic, counter); if (currentCounter != null) {counter = currntCounter}

Copy link
Contributor Author

Choose a reason for hiding this comment

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

good catch.
I was aware of the non-thread safe over first few calls, but did not realize the fix.
thanks.

}
return counter.getAndIncrement();
}

public void close() {}

}
Expand Up @@ -16,10 +16,15 @@
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.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.CountDownLatch;

import static java.util.Arrays.asList;
import static org.junit.Assert.assertEquals;
Expand Down Expand Up @@ -62,4 +67,81 @@ 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));
}

@Test
public void testPartitionPerf() throws Exception {
Copy link
Contributor

Choose a reason for hiding this comment

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

We should not include this along with the unit tests since it's not a unit test.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I am fine to delete it, where do you guys usually put such perf test codes to test method performance?

List<PartitionInfo> allPartitions = new ArrayList<PartitionInfo>();

final int topicCount = 37;
for (int i = 0; i < topicCount; ++i) {
final String topici = "topic" + i;
allPartitions.add(new PartitionInfo(topici, 0, node0, nodes, nodes));
allPartitions.add(new PartitionInfo(topici, 1, node1, nodes, nodes));
allPartitions.add(new PartitionInfo(topici, 2, node2, nodes, nodes));
}
final Cluster testCluster = new Cluster("clusterId", asList(node0, node1, node2), allPartitions,
Collections.<String>emptySet(), Collections.<String>emptySet());

int threadCount = 10;
final List<Long> partitionTimeList = Collections.synchronizedList(new ArrayList<Long>(threadCount));

final CountDownLatch latch = new CountDownLatch(threadCount);

final int loopCount = 1000000;
for (int i = 0; i < threadCount; ++i) {
new Thread(new Runnable() {
@Override
public void run() {
long start = System.currentTimeMillis();
for (int j = 0; j < loopCount; ++j) {
String theTopic = "topic" + j % topicCount;
partitioner.partition(theTopic, null, null, null, null, testCluster);
}
partitionTimeList.add(System.currentTimeMillis() - start);
latch.countDown();
}
}).start();
}

latch.await();

double sum = 0.0;
for (int i = 0; i < partitionTimeList.size(); ++i) {
sum += partitionTimeList.get(i);
}

double avg = sum / (double) loopCount;
System.out.println("avg partition time(ms)=" + String.format("%5f", avg));
Copy link
Contributor

Choose a reason for hiding this comment

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

Is this line intentional? Unit tests normally don't need to print out to console.

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, since this method is just to get an idea of the performance of Partition.partition method, something like method benchmark

}
}