[ https://issues.apache.org/jira/browse/KAFKA-6432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16535948#comment-16535948 ]
Ying Zheng commented on KAFKA-6432: ----------------------------------- The boxes in the diagrams are pages, rather than index entries. We can't guarantee the last 1 or 2 pages are always in the cache. It depends on the workload and the memory size of the host. The idea is to make the index look up logic more cache friendly. In any situation, the last a couple of pages are more likely to be in the cache, then the "median" page, as we always append at the end of the index. We actually do not even care about the page size. We just need to know that, a small section at the end of the index is more likely to be in the page cache than the median of index. And most of the index lookups (either from follower brokers or from in-sync consumers) should be looking up within this small section. In the code diff, i set the small warm section to be about 4KB, which is about smallest page size for most computer systems today. (https://en.wikipedia.org/wiki/Page_(computer_memory)#Multiple_page_sizes) 4KB index correspond to about 1.2MB (4KB / 12 bytes * 4KB) to 2MB (4KB / 8 bytes * 4KB, for offset index) log messages. In most cases , the gap between the leader and the in-sync followers / consumers should be larger than this number. > 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 > Priority: Major > 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)