[ https://issues.apache.org/jira/browse/KAFKA-6432?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Manikumar resolved KAFKA-6432. ------------------------------ Resolution: Fixed Fix Version/s: 2.1.0 > Lookup indices may cause unnecessary page fault > ----------------------------------------------- > > Key: KAFKA-6432 > URL: https://issues.apache.org/jira/browse/KAFKA-6432 > Project: Kafka > Issue Type: Improvement > Components: core, log > Reporter: Ying Zheng > Assignee: Ying Zheng > Priority: Major > Fix For: 2.1.0 > > Attachments: Binary Search - Diagram 1.png, Binary Search - Diagram > 2.png > > > For each topic-partition, Kafka broker maintains two indices: one for message > offset, one for message timestamp. By default, a new index entry is appended > to each index for every 4KB messages. The lookup of the indices is a simple > binary search. The indices are mmaped files, and cached by Linux page cache. > Both consumer fetch and follower fetch have to do an offset lookup, before > accessing the actual message data. The simple binary search algorithm used > for looking up the index is not cache friendly, and may cause page faults > even on high QPS topic-partitions. > For example (diagram 1), when looking up an index entry in page 12, the > binary search algorithm has to read page 0, 6, 9 and 11. After new messages > are appended to the topic-partition, the index grows to 13 pages. Now, if the > follower fetch request looking up the 1st index entry of page 13, the binary > search algorithm will go to page 0, 7, 10 and 12. Among those pages, page 7 > and 10 have not been used for a long time, and may already be swapped to hard > disk. > Actually, in a normal Kafka broker, all the follower fetch requests and most > consumer fetch requests should only look up the last few entries of the > index. We can make the index lookup more cache friendly, by searching in the > last one or two pages of the index first. (Diagram 2) -- This message was sent by Atlassian JIRA (v7.6.3#76005)