-
Notifications
You must be signed in to change notification settings - Fork 15.2k
KAFKA-19193 Support rack-aware partitioning for Kafka producer #19850
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
Merged
viktorsomogyi
merged 1 commit into
apache:trunk
from
ivanyu:kip-1123-rack-aware-partitioning
Apr 22, 2026
Merged
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or 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
This file contains hidden or 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
This file contains hidden or 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 |
|---|---|---|
|
|
@@ -28,6 +28,7 @@ | |
| import java.util.concurrent.ThreadLocalRandom; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import java.util.concurrent.atomic.AtomicReference; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| /** | ||
| * Built-in default partitioner. Note, that this is just a utility class that is used directly from | ||
|
|
@@ -40,8 +41,10 @@ public class BuiltInPartitioner { | |
| private final Logger log; | ||
| private final String topic; | ||
| private final int stickyBatchSize; | ||
| private final boolean rackAware; | ||
| private final String rack; | ||
|
|
||
| private volatile PartitionLoadStats partitionLoadStats = null; | ||
| private volatile PartitionLoadStatsHolder partitionLoadStatsHolder = null; | ||
| private final AtomicReference<StickyPartitionInfo> stickyPartitionInfo = new AtomicReference<>(); | ||
|
|
||
|
|
||
|
|
@@ -51,13 +54,15 @@ public class BuiltInPartitioner { | |
| * @param topic The topic | ||
| * @param stickyBatchSize How much to produce to partition before switch | ||
| */ | ||
| public BuiltInPartitioner(LogContext logContext, String topic, int stickyBatchSize) { | ||
| public BuiltInPartitioner(LogContext logContext, String topic, int stickyBatchSize, boolean rackAware, String rack) { | ||
| this.log = logContext.logger(BuiltInPartitioner.class); | ||
| this.topic = topic; | ||
| if (stickyBatchSize < 1) { | ||
| throw new IllegalArgumentException("stickyBatchSize must be >= 1 but got " + stickyBatchSize); | ||
| } | ||
| this.stickyBatchSize = stickyBatchSize; | ||
| this.rackAware = rackAware; | ||
| this.rack = rack; | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -67,14 +72,25 @@ private int nextPartition(Cluster cluster) { | |
| int random = randomPartition(); | ||
|
|
||
| // Cache volatile variable in local variable. | ||
| PartitionLoadStats partitionLoadStats = this.partitionLoadStats; | ||
| PartitionLoadStatsHolder partitionLoadStats = this.partitionLoadStatsHolder; | ||
|
|
||
| int partition; | ||
|
|
||
| if (partitionLoadStats == null) { | ||
| // We don't have stats to do adaptive partitioning (or it's disabled), just switch to the next | ||
| // partition based on uniform distribution. | ||
| List<PartitionInfo> availablePartitions = cluster.availablePartitionsForTopic(topic); | ||
| if (!availablePartitions.isEmpty()) { | ||
| // Select only partitions with leaders in this rack if configured so, falling back if none are available. | ||
| if (rackAware) { | ||
| List<PartitionInfo> availablePartitionsInRack = availablePartitions.stream() | ||
| .filter(p -> p.leader().hasRack() && p.leader().rack().equals(rack)) | ||
| .collect(Collectors.toList()); | ||
| if (!availablePartitionsInRack.isEmpty()) { | ||
| availablePartitions = availablePartitionsInRack; | ||
| } | ||
| } | ||
|
|
||
| partition = availablePartitions.get(random % availablePartitions.size()).partition(); | ||
| } else { | ||
| // We don't have available partitions, just pick one among all partitions. | ||
|
|
@@ -84,14 +100,20 @@ private int nextPartition(Cluster cluster) { | |
| } else { | ||
| // Calculate next partition based on load distribution. | ||
| // Note that partitions without leader are excluded from the partitionLoadStats. | ||
| assert partitionLoadStats.length > 0; | ||
|
|
||
| int[] cumulativeFrequencyTable = partitionLoadStats.cumulativeFrequencyTable; | ||
| int weightedRandom = random % cumulativeFrequencyTable[partitionLoadStats.length - 1]; | ||
| PartitionLoadStats partitionLoadStatsToUse = partitionLoadStats.total; | ||
| if (rackAware && partitionLoadStats.inThisRack != null && partitionLoadStats.inThisRack.length > 0) { | ||
| partitionLoadStatsToUse = partitionLoadStats.inThisRack; | ||
| } | ||
|
|
||
| assert partitionLoadStatsToUse.length > 0; | ||
|
|
||
| int[] cumulativeFrequencyTable = partitionLoadStatsToUse.cumulativeFrequencyTable; | ||
| int weightedRandom = random % cumulativeFrequencyTable[partitionLoadStatsToUse.length - 1]; | ||
|
|
||
| // By construction, the cumulative frequency table is sorted, so we can use binary | ||
| // search to find the desired index. | ||
| int searchResult = Arrays.binarySearch(cumulativeFrequencyTable, 0, partitionLoadStats.length, weightedRandom); | ||
| int searchResult = Arrays.binarySearch(cumulativeFrequencyTable, 0, partitionLoadStatsToUse.length, weightedRandom); | ||
|
|
||
| // binarySearch results the index of the found element, or -(insertion_point) - 1 | ||
| // (where insertion_point is the index of the first element greater than the key). | ||
|
|
@@ -103,8 +125,8 @@ private int nextPartition(Cluster cluster) { | |
| // would return -0 - 1 = -1, by adding 1 we'd get 0. If we're looking for 4, we'd | ||
| // get 0, and we need the next one, so adding 1 works here as well. | ||
| int partitionIndex = Math.abs(searchResult + 1); | ||
| assert partitionIndex < partitionLoadStats.length; | ||
| partition = partitionLoadStats.partitionIds[partitionIndex]; | ||
| assert partitionIndex < partitionLoadStatsToUse.length; | ||
| partition = partitionLoadStatsToUse.partitionIds[partitionIndex]; | ||
| } | ||
|
|
||
| log.trace("Switching to partition {} in topic {}", partition, topic); | ||
|
|
@@ -120,9 +142,15 @@ int randomPartition() { | |
| * random number. | ||
| */ | ||
| public int loadStatsRangeEnd() { | ||
| assert partitionLoadStats != null; | ||
| assert partitionLoadStats.length > 0; | ||
| return partitionLoadStats.cumulativeFrequencyTable[partitionLoadStats.length - 1]; | ||
| assert partitionLoadStatsHolder != null; | ||
| assert partitionLoadStatsHolder.total.length > 0; | ||
| return partitionLoadStatsHolder.total.cumulativeFrequencyTable[partitionLoadStatsHolder.total.length - 1]; | ||
| } | ||
|
|
||
| public int loadStatsInThisRackRangeEnd() { | ||
| assert partitionLoadStatsHolder.inThisRack != null; | ||
| assert partitionLoadStatsHolder.inThisRack.length > 0; | ||
| return partitionLoadStatsHolder.inThisRack.cumulativeFrequencyTable[partitionLoadStatsHolder.inThisRack.length - 1]; | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -233,18 +261,20 @@ void updatePartitionInfo(StickyPartitionInfo partitionInfo, int appendedBytes, C | |
| * | ||
| * @param queueSizes The queue sizes, partitions without leaders are excluded | ||
| * @param partitionIds The partition ids for the queues, partitions without leaders are excluded | ||
| * @param partitionLeaderRacks The racks of partition leaders for the queues, partitions without leaders are excluded | ||
| * @param length The logical length of the arrays (could be less): we may eliminate some partitions | ||
| * based on latency, but to avoid reallocation of the arrays, we just decrement | ||
| * logical length | ||
| * Visible for testing | ||
| */ | ||
| public void updatePartitionLoadStats(int[] queueSizes, int[] partitionIds, int length) { | ||
| public void updatePartitionLoadStats(int[] queueSizes, int[] partitionIds, String[] partitionLeaderRacks, int length) { | ||
| if (queueSizes == null) { | ||
| log.trace("No load stats for topic {}, not using adaptive", topic); | ||
| partitionLoadStats = null; | ||
| partitionLoadStatsHolder = null; | ||
| return; | ||
| } | ||
| assert queueSizes.length == partitionIds.length; | ||
| assert queueSizes.length == partitionLeaderRacks.length; | ||
| assert length <= queueSizes.length; | ||
|
|
||
| // The queueSizes.length represents the number of all partitions in the topic and if we have | ||
|
|
@@ -257,7 +287,7 @@ public void updatePartitionLoadStats(int[] queueSizes, int[] partitionIds, int l | |
| if (length < 1 || queueSizes.length < 2) { | ||
| log.trace("The number of partitions is too small: available={}, all={}, not using adaptive for topic {}", | ||
| length, queueSizes.length, topic); | ||
| partitionLoadStats = null; | ||
| partitionLoadStatsHolder = null; | ||
| return; | ||
| } | ||
|
|
||
|
|
@@ -276,6 +306,7 @@ public void updatePartitionLoadStats(int[] queueSizes, int[] partitionIds, int l | |
| // the value is the index of the partition we're looking for. In this example | ||
| // random numbers 0, 1, 2, 3 would map to partition[0], 4 would map to partition[1] | ||
| // and 5, 6, 7 would map to partition[2]. | ||
| // Do the same with this-rack-only partitions if rack awareness is enabled. | ||
|
|
||
| // Calculate max queue size + 1 and check if all sizes are the same. | ||
| int maxSizePlus1 = queueSizes[0]; | ||
|
|
@@ -293,18 +324,55 @@ public void updatePartitionLoadStats(int[] queueSizes, int[] partitionIds, int l | |
| // and we didn't exclude partitions that experience high latencies (greater than | ||
| // partitioner.availability.timeout.ms). | ||
| log.trace("All queue lengths are the same, not using adaptive for topic {}", topic); | ||
| partitionLoadStats = null; | ||
| partitionLoadStatsHolder = null; | ||
| return; | ||
| } | ||
|
|
||
| // Before inverting and folding, build fully the load stats for this rack, because this depends on the raw queue sizes. | ||
| PartitionLoadStats partitionLoadStatsInThisRack = createPartitionLoadStatsForThisRackIfNeeded(queueSizes, partitionIds, partitionLeaderRacks, length); | ||
|
|
||
| // Invert and fold the queue size, so that they become separator values in the CFT. | ||
| invertAndFoldQueueSizeArray(queueSizes, maxSizePlus1, length); | ||
|
|
||
| log.trace("Partition load stats for topic {}: CFT={}, IDs={}, length={}", | ||
|
Member
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. Should we include the rack-specific load stats (partitionLoadStatsInThisRack) in this trace log when
Contributor
Author
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. Please have a look: #22434 |
||
| topic, queueSizes, partitionIds, length); | ||
| partitionLoadStatsHolder = new PartitionLoadStatsHolder( | ||
| new PartitionLoadStats(queueSizes, partitionIds, length), | ||
| partitionLoadStatsInThisRack | ||
| ); | ||
| } | ||
|
|
||
| private PartitionLoadStats createPartitionLoadStatsForThisRackIfNeeded(int[] queueSizes, int[] partitionIds, String[] partitionLeaderRacks, int length) { | ||
| if (!rackAware) { | ||
| return null; | ||
| } | ||
| int[] queueSizesInThisRack = new int[length]; | ||
| int[] partitionIdsInThisRack = new int[length]; | ||
| int lengthInThisRack = 0; | ||
| int maxSizePlus1InThisRack = -1; | ||
|
|
||
| for (int i = 0; i < length; i++) { | ||
| if (rack.equals(partitionLeaderRacks[i])) { | ||
| queueSizesInThisRack[lengthInThisRack] = queueSizes[i]; | ||
| partitionIdsInThisRack[lengthInThisRack] = partitionIds[i]; | ||
|
|
||
| if (queueSizes[i] > maxSizePlus1InThisRack) | ||
| maxSizePlus1InThisRack = queueSizes[i]; | ||
|
|
||
| lengthInThisRack += 1; | ||
| } | ||
| } | ||
| ++maxSizePlus1InThisRack; | ||
|
|
||
| invertAndFoldQueueSizeArray(queueSizesInThisRack, maxSizePlus1InThisRack, lengthInThisRack); | ||
| return new PartitionLoadStats(queueSizesInThisRack, partitionIdsInThisRack, lengthInThisRack); | ||
| } | ||
|
|
||
| private void invertAndFoldQueueSizeArray(int[] queueSizes, int maxSizePlus1, int length) { | ||
|
viktorsomogyi marked this conversation as resolved.
|
||
| queueSizes[0] = maxSizePlus1 - queueSizes[0]; | ||
| for (int i = 1; i < length; i++) { | ||
| queueSizes[i] = maxSizePlus1 - queueSizes[i] + queueSizes[i - 1]; | ||
| } | ||
| log.trace("Partition load stats for topic {}: CFT={}, IDs={}, length={}", | ||
| topic, queueSizes, partitionIds, length); | ||
| partitionLoadStats = new PartitionLoadStats(queueSizes, partitionIds, length); | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -346,4 +414,15 @@ public PartitionLoadStats(int[] cumulativeFrequencyTable, int[] partitionIds, in | |
| this.length = length; | ||
| } | ||
| } | ||
|
|
||
| private static final class PartitionLoadStatsHolder { | ||
| final PartitionLoadStats total; | ||
| final PartitionLoadStats inThisRack; | ||
|
|
||
| private PartitionLoadStatsHolder(PartitionLoadStats total, | ||
| PartitionLoadStats inThisRack) { | ||
| this.total = total; | ||
| this.inThisRack = inThisRack; | ||
| } | ||
| } | ||
| } | ||
Oops, something went wrong.
Oops, something went wrong.
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.
Could we add the new params in the javadoc?
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.
@junrao sure, please have a look: #22432