-
Notifications
You must be signed in to change notification settings - Fork 13.6k
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
Changes from 9 commits
d74b0bd
fdd946a
f978b50
f9ca411
82421e4
59a5298
6b99f6e
c5d0bb0
51eed89
b560d1b
eb8d1bd
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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; | ||
|
@@ -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) {} | ||
|
||
|
@@ -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 = getNextValue(topic); | ||
List<PartitionInfo> availablePartitions = cluster.availablePartitionsForTopic(topic); | ||
if (availablePartitions.size() > 0) { | ||
int part = Utils.toPositive(nextValue) % availablePartitions.size(); | ||
|
@@ -68,6 +70,18 @@ public int partition(String topic, Object key, byte[] keyBytes, Object value, by | |
} | ||
} | ||
|
||
private int getNextValue(String topic) { | ||
AtomicInteger counter = topicCounterMap.get(topic); | ||
if (null == counter) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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)? There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. 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); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 |
---|---|---|
|
@@ -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; | ||
|
@@ -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 { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. There was a problem hiding this comment. Choose a reason for hiding this commentThe 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<Double> avgPartitionTimeList = Collections.synchronizedList(new ArrayList<Double>(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); | ||
} | ||
avgPartitionTimeList.add((double) (System.currentTimeMillis() - start) / (double) loopCount); | ||
latch.countDown(); | ||
} | ||
}).start(); | ||
} | ||
|
||
latch.await(); | ||
|
||
double sum = 0.0; | ||
for (int i = 0; i < avgPartitionTimeList.size(); ++i) { | ||
sum += avgPartitionTimeList.get(i); | ||
} | ||
|
||
double avg = sum / (double) avgPartitionTimeList.size(); | ||
System.out.println("avg partition time(ms)=" + String.format("%5f", avg)); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 |
||
} | ||
} |
There was a problem hiding this comment.
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. SonextValue
would probably be better.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yeah, updated.