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

Jason Gustafson commented on KAFKA-10888:
-----------------------------------------

We have found one cause of imbalance when the sticky partitioner is used. 
Basically the intuition behind the sticky partitioner breaks down a little bit 
when a small `linger.ms` is in use (sadly this is the default). The user is 
sort of opting out of batching with this setting which means there is little 
opportunity to fill batches before they get drained and sent. That leaves the 
door open for a kind of imbalanced write problem.

To see why, suppose that we have a producer writing to 3 partitions with 
linger.ms=0 and one partition slows down a little bit for some reason. It could 
be a leader change or some transient network issue. The producer will have to 
hold onto the batches for that partition until it becomes available. While it 
is holding onto those batches, additional batches will begin piling up. Each of 
these batches is likely to get filled because the producer is not ready to send 
to this partition yet.

Consider this from the perspective of the sticky partitioner. Every time the 
slow partition gets selected, the producer will fill the batches completely. On 
the other hand, the remaining "fast" partitions will likely not get their 
batches filled because of the `linger.ms=0` setting. As soon as a single record 
is available, it might get sent. So more data ends up getting written to the 
partition that has already started to build a backlog. And even after the cause 
of the original slowness (e.g. leader change) gets resolved, it might take some 
time for this imbalance to recover. We believe this can even create a runaway 
effect if the partition cannot catch up with the handicap of the additional 
load.

We analyzed one case where we thought this might be going on. Below I've 
summarized the writes over a period of one hour to 3 partitions. Partition 0 
here is the "slow" partition. All partitions get roughly the same number of 
batches, but the slow partition has much bigger batch sizes.

{code}
Partition TotalBatches TotalBytes TotalRecords BytesPerBatch RecordsPerBatch
0         1683         25953200   25228        15420.80      14.99        
1         1713         7836878    4622         4574.94       2.70
2         1711         7546212    4381         4410.41       2.56
{code}

After restarting the application, the producer was healthy again. It just was 
not able to recover with the imbalanced workload.

>  Sticky partition leads to uneven product msg, resulting in abnormal delays 
> in some partations
> ----------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-10888
>                 URL: https://issues.apache.org/jira/browse/KAFKA-10888
>             Project: Kafka
>          Issue Type: Bug
>          Components: clients, producer 
>    Affects Versions: 2.4.1
>            Reporter: jr
>            Priority: Major
>         Attachments: image-2020-12-24-21-05-02-800.png, 
> image-2020-12-24-21-09-47-692.png, image-2020-12-24-21-10-24-407.png
>
>
>   110 producers ,550 partitions ,550 consumers , 5 nodes Kafka cluster
>   The producer uses the nullkey+stick partitioner, the total production rate 
> is about 100w tps
> Observed partition delay is abnormal and message distribution is uneven, 
> which leads to the maximum production and consumption delay of the partition 
> with more messages 
> abnormal.
>   I cannot find reason that stick will make the message distribution uneven 
> at this production rate.
>   I can't switch to the round-robin partitioner, which will increase the 
> delay and cpu cost. Is thathe stick partationer design cause uneven message 
> distribution, or this is abnormal. How to solve it?
>   !image-2020-12-24-21-09-47-692.png!
> As shown in the picture, the uneven distribution is concentrated on some 
> partitions and some brokers, there seems to be some rules.
> This problem does not only occur in one cluster, but in many high tps 
> clusters,
> The problem is more obvious on the test cluster we built.
> !image-2020-12-24-21-10-24-407.png!



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to