[ https://issues.apache.org/jira/browse/KAFKA-8106?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16845497#comment-16845497 ]
ASF GitHub Bot commented on KAFKA-8106: --------------------------------------- guozhangwang commented on pull request #6785: KAFKA-8106: Skipping ByteBuffer allocation of key / value / headers in logValidator URL: https://github.com/apache/kafka/pull/6785 This is based on @Flowermin in https://github.com/apache/kafka/pull/6699 addressing the comments. 1. Add skipKeyValueIterator() used in LogValidator; based on wether `isCompressed` and `skipKeyValue` there are four possible iterators now. 2. Add SkipKeyValueDefaultRecord which extends DefaultRecord. ------------- We suggest that reducing the allocation and copying of ByteBuffer when logValidator do validation when magic value to use is above 1 and no format conversion or value overwriting is required for compressed messages.And improved code is as follows. 1. Adding a class **SimplifiedDefaultRecord** implement class Record which define various attributes of a message. 2. Adding Function **simplifiedreadFrom**() at class **DefaultRecord** .This function will not read data from DataInput to ByteBuffer which need newly creating .**This will reduce the allocation and copying of ByteBuffer** when logValidator do validation .This will reduces GC frequency. We offer a simple read function to read data from **DataInput** whithout create ByteBuffer.Of course this opertion can not avoid deconmpression to data. 3. Adding Function **simplifiedIterator**() and **simplifiedCompressedIterator**() at class **DefaultRecordBatch**.This two functions will return iterator of instance belongs to class **SimplifiedDefaultRecord**. 4. Modify code of function **validateMessagesAndAssignOffsetsCompressed**() at class LogValidator. **After modifing code wich reducing the allocation and copying of ByteBuffer**, the test performance is greatly improved, and the CPU's stable usage is below 60%. The following is a comparison of different code test performance under the same conditions. **Result of performance testing** Main config of Kafka: Single Message:1024B;TopicPartitions:200;linger.ms:1000ms, **1.Before modified code(Source code):** Network inflow rate:600M/s;CPU(%)(97%);production:25,000,000 messages/s **2.After modified code(remove allocation of ByteBuffer):** Network inflow rate:1G/s;CPU(%)(<60%);production:41,000,000 messages/s **1.Before modified code(Source code) GC:**  **2.After modified code(remove allocation of ByteBuffer) GC:**  ### Committer Checklist (excluded from commit message) - [ ] Verify design and implementation - [ ] Verify test coverage and CI build status - [ ] Verify documentation (including upgrade notes) ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org > Reducing the allocation and copying of ByteBuffer 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.2.0, 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 > .After we checked and completed the performance test again, we located the > code "*ByteBuffer recordBuffer = > ByteBuffer.allocate(sizeOfBodyInBytes);*(*Class:DefaultRecord,Function:readFrom()*)” > which consumed CPU resources and caused a lot of GC .Our modified code > reduces the allocation and copying of ByteBuffer, so the test performance is > greatly improved, and the CPU's stable usage is *below 60%*. The following is > a comparison of different code test performance under the same conditions. > *Result of performance testing* > *Main config of Kafka: Single > Message:1024B;TopicPartitions:200;linger.ms:1000ms.* > | Single Message : 1024B,|Network inflow rate|CPU(%)|Messages/s| > |Source code|600M/s|97%|25,000,000| > |Modified code|1GB/s|<60%|41,660,000| > **1.Before modified code(Source code) GC:** >  > **2.After modified code(remove allocation of ByteBuffer) GC:** >  -- This message was sent by Atlassian JIRA (v7.6.3#76005)