[ https://issues.apache.org/jira/browse/KAFKA-3777?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15313382#comment-15313382 ]
Jay Kreps commented on KAFKA-3777: ---------------------------------- Probably everyone already knows but just want to capture a couple of goals of that cache originally: 1. Avoid serialization and deserialization with the read AND write to/from RocksDB 2. Avoid the write to kafka and rocksdb for duplicate updates 3. Allow doing larger batch writes to rocksdb which seemed to significantly cut down on overhead A long time ago in Samza we had done some benchmarking of each of these. At that time (3) was a big across the board impact, and (1) and (2) really depend on the cachability of the read and write stream. I also think that this was done originally with leveldb and then an early version of rocksdb and many things may have changed since then, particularly, (a) I am not sure if the batch write thing matters any more, (b) I think they added a way to disable logging entirely which should be a big win as long as we handle recovery from kafka in cases of unclean shutdown. > Extract the LRU cache out of RocksDBStore > ----------------------------------------- > > Key: KAFKA-3777 > URL: https://issues.apache.org/jira/browse/KAFKA-3777 > Project: Kafka > Issue Type: Sub-task > Components: streams > Affects Versions: 0.10.1.0 > Reporter: Eno Thereska > Fix For: 0.10.1.0 > > > The LRU cache that is currently inside the RocksDbStore class. As part of > KAFKA-3776 it needs to come outside of RocksDbStore and be a separate > component used in: > 1. KGroupedStream.aggregate() / reduce(), > 2. KStream.aggregateByKey() / reduceByKey(), > 3. KTable.to() (this will be done in KAFKA-3779). > As all of the above operators can have a cache on top to deduplicate the > materialized state store in RocksDB. > The scope of this JIRA is to extract out the cache of RocksDBStore, and keep > them as item 1) and 2) above; and it should be done together / after > KAFKA-3780. -- This message was sent by Atlassian JIRA (v6.3.4#6332)