KAFKA-19193 Support rack-aware partitioning for Kafka producer#19850
KAFKA-19193 Support rack-aware partitioning for Kafka producer#19850ivanyu wants to merge 1 commit intoapache:trunkfrom
Conversation
1af9189 to
564e60f
Compare
|
A label of 'needs-attention' was automatically added to this PR in order to raise the |
|
Is there any chance this feature will be included in the next release? |
|
@marcospassos I'm waiting for 4.1 release to be cut, after that I'll ping the mailing list for review. |
|
A label of 'needs-attention' was automatically added to this PR in order to raise the |
|
A label of 'needs-attention' was automatically added to this PR in order to raise the |
| private final String rack; | ||
|
|
||
| private volatile PartitionLoadStats partitionLoadStats = null; | ||
| private volatile PartitionLoadStatsHolder partitionLoadStats = null; |
There was a problem hiding this comment.
| private volatile PartitionLoadStatsHolder partitionLoadStats = null; | |
| private volatile PartitionLoadStatsHolder partitionLoadStatsHolder = null; |
As it got a bit confusing when later the code is retrieving partitionLoadStats.inThisRack which is a member of PartitionLoadStatsHolder of type PartitionLoadStats. :D
|
|
||
| // Create partitions with "sticky" batch size to accommodate 3 records. | ||
| BuiltInPartitioner builtInPartitionerA = new SequentialPartitioner(logContext, TOPIC_A, 3); | ||
| BuiltInPartitioner builtInPartitionerA = new SequentialPartitioner(logContext, TOPIC_A, 3, false, ""); |
There was a problem hiding this comment.
| BuiltInPartitioner builtInPartitionerA = new SequentialPartitioner(logContext, TOPIC_A, 3, false, ""); | |
| final boolean rackAware = false; | |
| final String clientRackId = ""; | |
| BuiltInPartitioner builtInPartitionerA = new SequentialPartitioner(logContext, TOPIC_A, 3, rackAware, clientRackId); |
|
|
||
| // Check that switching works even when there is one partition. | ||
| BuiltInPartitioner builtInPartitionerB = new SequentialPartitioner(logContext, TOPIC_B, 1); | ||
| BuiltInPartitioner builtInPartitionerB = new SequentialPartitioner(logContext, TOPIC_B, 1, false, ""); |
There was a problem hiding this comment.
| BuiltInPartitioner builtInPartitionerB = new SequentialPartitioner(logContext, TOPIC_B, 1, false, ""); | |
| BuiltInPartitioner builtInPartitionerB = new SequentialPartitioner(logContext, TOPIC_B, 1, rackAware, clientRackId); |
Just to make the meaning of false and "" obvious at a glance :)
| partitionRacks[i] = leader.rack(); | ||
| } | ||
| allPartitions.add(new PartitionInfo(TOPIC_A, i, leader, NODES, NODES)); | ||
| expectedFrequencies[i] = 6 - queueSizes[i]; // 6 is max(queueSizes) + 1 |
|
This PR is being marked as stale since it has not had any activity in 90 days. If you If you are having difficulty finding a reviewer, please reach out on the [mailing list](https://kafka.apache.org/contact). If this PR is no longer valid or desired, please feel free to close it. If no activity occurs in the next 30 days, it will be automatically closed. |
|
This PR has been closed since it has not had any activity in 120 days. If you feel like this |
|
Hi, unfortunately this was closed. Are there any plan to continue this effort or anything we could do to support it? |
|
This issue would definitely benefit the community and should not be closed. |
|
@ivanyu—are you still able to work on this? Or do you need the community to follow up? LMK. Thanks! |
|
@kirktrue sure, I will review the PR soon! |
|
For some reason, I cannot reopen the PR (the button isn't even there). Maybe someone else can? |
564e60f to
6816a21
Compare
|
Thank you @LiamClarkeNZ, I addressed your comments |
According to KIP-1123, this commit adds the support for rack-aware partitioning to `BuiltInPartitioner`. It comes with two new configs for the producer: `partitioner.rack.aware` and `client.rack`, which allows enabling the new behavior. Apart from the added unit tests, the desired behavior was validated by `kafka-producer-perf-test.sh` with an existing and a non-existing rack against a 4 node cluster with two racks and 12-partition topic: ```shell ./kafka_2.13-4.1.0-SNAPSHOT/bin/kafka-producer-perf-test.sh \ --topic test-topic --num-records 100000 --throughput -1 --record-size 1 \ --producer-props bootstrap.servers=127.0.0.1:9092 \ client.rack=rack0 partitioner.rack.aware=true ```
6816a21 to
605e049
Compare
According to KIP-1123, this commit adds the support for rack-aware partitioning to
BuiltInPartitioner. It comes with two new configs for the producer:partitioner.rack.awareandclient.rack, which allows enabling the new behavior.Apart from the added unit tests, the desired behavior was tested by
kafka-producer-perf-test.shwith an existing and a non-existing rack against a 4 node cluster with two racks and 12-partition topic:Delete this text and replace it with a detailed description of your change. The
PR title and body will become the squashed commit message.
If you would like to tag individuals, add some commentary, upload images, or
include other supplemental information that should not be part of the eventual
commit message, please use a separate comment.
If applicable, please include a summary of the testing strategy (including
rationale) for the proposed change. Unit and/or integration tests are expected
for any behavior change and system tests should be considered for larger
changes.