[ 
https://issues.apache.org/jira/browse/KAFKA-17632?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17893559#comment-17893559
 ] 

Daniel Urban commented on KAFKA-17632:
--------------------------------------

Thank you for the report and the repro code.

I'm unsure if this ever worked correctly, but in the current code, 
KafkaProducer calls partitioner.partition twice on new batches, this is the 
location of the 2nd call:

[https://github.com/apache/kafka/blob/14a9130f6fa31c10cb65cc500c101148d0410306/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java#L1079]

Because of this, when you have an even number of partitions, you are basically 
"locked out" of half of your partitions when using RR.

RoundRobinPartitioner was implemented without taking the semantics of the 
onNewBatch method into account. I'll try to submit a fix for this - basically 
the onNewBatch can be handled as a "do not increment on the next call of 
partition" notification.

I think the only way to fix your custom partitioner implementation is to 
introduce similar logic in the onNewBatch callback. Since onNewBatch is 
deprecated, it will be removed in the future, and that will probably fix the 
logic in KafkaProducer. Due to backward compatibility, and other partitioner 
implementations relying on this double-call, I think we cannot really change 
the producer logic.

> Custom `partitioner.class` with an even number of partitions always writes to 
> even partitions if use RoundRobinPartitioner
> --------------------------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-17632
>                 URL: https://issues.apache.org/jira/browse/KAFKA-17632
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 3.8.0
>            Reporter: thanhlv
>            Assignee: Daniel Urban
>            Priority: Minor
>         Attachments: image-2024-09-27-15-05-53-707.png
>
>
> Our project has some special logic that requires custom partitions.
> With an odd number of Partitions, everything works fine and well.
> However, with an even number of partitions. Data will only be written to 
> even-numbered Partition IDs
> Info:
> Lib Java: `kafka-clients:3.8.0`
> Code demo:
> {code:java}
> public class CustomLogicPartitionMain {
>     public static void main(String[] args) throws IOException {
>         System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "ALL");
>         final var props = new Properties();
>         props.setProperty(ProducerConfig.CLIENT_ID_CONFIG, 
> "java-producer-producerRecordPartition-KeyNotNull");
>         props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, 
> "localhost:29091,localhost:29092,localhost:29093,localhost:29094");
>         props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, 
> StringSerializer.class.getName());
>         props.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, 
> StringSerializer.class.getName());
>         props.setProperty(ProducerConfig.BATCH_SIZE_CONFIG, "5000");
>         props.setProperty(ProducerConfig.PARTITIONER_CLASS_CONFIG, 
> "org.apache.kafka.clients.producer.RoundRobinPartitioner");
>         try (var producer = new KafkaProducer<Object, String>(props)) {
>             final var messageProducerRecord = new ProducerRecord<>(
>                     "topic-rep-1-partition-10",     //topic name
>                     // 36 byte
>                     UUID.randomUUID().toString()        // value
>             );
>             for (int i = 1; i <= 5000; i++) {
>                 producer.send(messageProducerRecord);
>             }
>         }
>     }
> }
>  {code}
> !image-2024-09-27-15-05-53-707.png!
>  
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to