You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Flower.min (JIRA)" <ji...@apache.org> on 2019/05/08 02:09:01 UTC

[jira] [Updated] (KAFKA-8106) Reduces the allocation and copying of ByteBuffer operation when logValidator do validation.

     [ https://issues.apache.org/jira/browse/KAFKA-8106?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Flower.min updated KAFKA-8106:
------------------------------
    Summary: Reduces the allocation and copying of ByteBuffer operation when logValidator  do validation.  (was: Remove unnecessary decompression operation when logValidator  do validation.)

> Reduces the allocation and copying of ByteBuffer operation when logValidator  do validation.
> --------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-8106
>                 URL: https://issues.apache.org/jira/browse/KAFKA-8106
>             Project: Kafka
>          Issue Type: Bug
>          Components: core
>    Affects Versions: 2.1.1
>         Environment: Server : 
> cpu:2*16 ; 
> MemTotal : 256G;
> Ethernet controller:Intel Corporation 82599ES 10-Gigabit SFI/SFP+ Network Connection ; 
> SSD.
>            Reporter: Flower.min
>            Assignee: Flower.min
>            Priority: Major
>              Labels: performance
>
>       We do performance testing about Kafka in specific scenarios as described below .We build a kafka cluster with one broker,and create topics with different number of partitions.Then we start lots of producer processes to send large amounts of messages to one of the topics at one  testing .
> *_Specific Scenario_*
>   
>  *_1.Main config of Kafka_*  
>  # Main config of Kafka  server:[num.network.threads=6;num.io.threads=128;queued.max.requests=500|http://num.network.threads%3D6%3Bnum.io.threads%3D128%3Bqueued.max.requests%3D500/]
>  # Number of TopicPartition : 50~2000
>  # Size of Single Message : 1024B
>  
>  *_2.Config of KafkaProducer_* 
> ||compression.type||[linger.ms|http://linger.ms/]||batch.size||buffer.memory||
> |lz4|1000ms~5000ms|16KB/10KB/100KB|128MB|
> *_3.The best result of performance testing_*  
> ||Network inflow rate||CPU Used (%)||Disk write speed||Performance of production||
> |550MB/s~610MB/s|97%~99%|:550MB/s~610MB/s       |23,000,000 messages/s|
> *_4.Phenomenon and  my doubt_*
>         _The upper limit of CPU usage has been reached  But  it does not reach the upper limit of the bandwidth of the server  network. *We are doubtful about which  cost too much CPU time and we want to Improve  performance and reduces CPU usage of Kafka server.*_
>   
>  _*5.Analysis*_
>         We analysis the JFIR of Kafka server when doing performance testing .We found the hot spot method is *_"java.io.DataInputStream.readFully(byte[],int,int)"_* and *_"org.apache.kafka.common.record.KafkaLZ4BlockInputStream.read(byte[],int,int)"_*.When  we checking thread stack information we  also have found most CPU being occupied by lots of thread  which  is busy decompressing messages.Then we read source code of Kafka .
>        There is double-layer nested Iterator  when LogValidator do validate every record.And There is a decompression for each message when traversing every RecordBatch iterator. It is consuming CPU and affect total performance that  decompress message._*The purpose of decompressing every messages just for gain total size in bytes of one record and size in bytes of record body when magic value to use is above 1 and no format conversion or value overwriting is required for compressed messages.It is negative for performance in common usage scenarios .*_{color:#333333}Therefore, we suggest that *_removing unnecessary decompression operation_* when doing  validation for compressed message  when magic value to use is above 1 and no format conversion or value overwriting is required for compressed messages.{color}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)