You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
KAFKA-19193 Support rack-aware partitioning for Kafka producer (#19850)
Implements [KIP-1123](https://cwiki.apache.org/confluence/display/KAFKA/KIP-1123%3A+Rack-aware+partitioning+for+Kafka+Producer).
This PR 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 tested 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
```
Reviewers: Viktor Somogyi-Vass <viktorsomogyi@gmail.com>, Liam Clarke-Hutchinson <liam@steelsky.co.nz>
privatestaticfinalStringPARTITIONER_RACK_AWARE_DOC = "Controls whether the default partitioner is rack-aware. This has no effect when a custom partitioner is used.";
128
+
125
129
/** <code>acks</code> */
126
130
publicstaticfinalStringACKS_CONFIG = "acks";
127
131
privatestaticfinalStringACKS_DOC = "The number of acknowledgments the producer requires the leader to have received before considering a request complete. This controls the "
@@ -178,6 +182,12 @@ public class ProducerConfig extends AbstractConfig {
@@ -317,7 +327,9 @@ public class ProducerConfig extends AbstractConfig {
317
327
"This strategy send records to a partition until at least " + BATCH_SIZE_CONFIG + " bytes is produced to the partition. It works with the strategy:" +
318
328
"<ol>" +
319
329
"<li>If no partition is specified but a key is present, choose a partition based on a hash of the key.</li>" +
320
-
"<li>If no partition or key is present, choose the sticky partition that changes when at least " + BATCH_SIZE_CONFIG + " bytes are produced to the partition.</li>" +
330
+
"<li>If no partition or key is present, choose the sticky partition that changes when at least <code>" + BATCH_SIZE_CONFIG + "</code> bytes are produced to the partition.</li>" +
331
+
"<li>If <code>" + CLIENT_RACK_CONFIG + "</code> is specified and <code>" + PARTITIONER_RACK_AWARE_CONFIG + "=true</code>, the sticky partition is chosen from partitions " +
332
+
"with the leader broker in the same rack, if at least one is available. If none are available, it falls back on selecting from all available partitions.</li>" +
321
333
"</ol>" +
322
334
"</li>" +
323
335
"<li><code>org.apache.kafka.clients.producer.RoundRobinPartitioner</code>: A partitioning strategy where " +
@@ -402,9 +414,11 @@ public class ProducerConfig extends AbstractConfig {
0 commit comments