-
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
Closed
yaojuncn
wants to merge
11
commits into
apache:trunk
from
yaojuncn:KAFKA-4402-client-producer-round-robin-fix
Closed
Changes from all commits
Commits
Show all changes
11 commits
Select commit
Hold shift + click to select a range
d74b0bd
KAFKA-4402: make the KafkaProducer true round robin per topic
yaojuncn fdd946a
KAFKA-4402: fix some style issues
yaojuncn f978b50
[KAFKA-4402] add performance test for round-robin default partitioner,
yaojuncn f9ca411
Merge branch 'trunk' into KAFKA-4402-client-producer-round-robin-fix
yaojuncn 82421e4
KAFKA-4402: fix some style issues in DefaultPartitionerTest.java
yaojuncn 59a5298
KAFKA-4402 fix DefaultPartitionerTest.testPartitionPerf time calculat…
yaojuncn 6b99f6e
KAFKA-4402 fix test code checkstyle error
yaojuncn c5d0bb0
KAFKA-4402, fix a bug in DefaultPartitionerTest.testPartitionPerf
yaojuncn 51eed89
KAFKA-4402: fix a race condition in DefaultPartitioner
yaojuncn b560d1b
KAFKA-4402 rename "getNextValue" to "nextValue", as this is kafka sytle
yaojuncn eb8d1bd
KAFKA-4402 remove perf test codes from unit test
yaojuncn File filter
Filter by extension
Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
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 = nextValue(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 nextValue(String topic) { | ||
AtomicInteger counter = topicCounterMap.get(topic); | ||
if (null == counter) { | ||
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() {} | ||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
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 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.