[ https://issues.apache.org/jira/browse/KAFKA-4212?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16882449#comment-16882449 ]
James Ritt commented on KAFKA-4212: ----------------------------------- Thanks [~ableegoldman] & [~mjsax]! So I took a look at [`RocksDBConfigSetter`|https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/RocksDBConfigSetter.java#L45] and my current understanding is that its `setConfig` method is used to mutate the provided `org.rocksdb.Options` before it's subsequently passed into the rocks DB constructor. Unfortunately, ttl doesn't seem to be configurable within `org.rocksdb.Options`, and instead, as far as I can tell, the usage of the TtlDB would instead need to be enacted by using the appropriate constructor [here|https://github.com/apache/kafka/blob/trunk/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java#L191]. Please let me know if I'm missing something, but given the above, the next approach I was considering was adding a StreamsConfig rocksDbTtl setting similar to [here;|https://github.com/apache/kafka/pull/2159/files] your thoughts? > Add a key-value store that is a TTL persistent cache > ---------------------------------------------------- > > Key: KAFKA-4212 > URL: https://issues.apache.org/jira/browse/KAFKA-4212 > Project: Kafka > Issue Type: Improvement > Components: streams > Affects Versions: 0.10.0.1 > Reporter: Elias Levy > Priority: Major > Labels: api > > Some jobs needs to maintain as state a large set of key-values for some > period of time. I.e. they need to maintain a TTL cache of values potentially > larger than memory. > Currently Kafka Streams provides non-windowed and windowed key-value stores. > Neither is an exact fit to this use case. > The {{RocksDBStore}}, a {{KeyValueStore}}, stores one value per key as > required, but does not support expiration. The TTL option of RocksDB is > explicitly not used. > The {{RocksDBWindowsStore}}, a {{WindowsStore}}, can expire items via segment > dropping, but it stores multiple items per key, based on their timestamp. > But this store can be repurposed as a cache by fetching the items in reverse > chronological order and returning the first item found. > KAFKA-2594 introduced a fixed-capacity in-memory LRU caching store, but here > we desire a variable-capacity memory-overflowing TTL caching store. > Although {{RocksDBWindowsStore}} can be repurposed as a cache, it would be > useful to have an official and proper TTL cache API and implementation. -- This message was sent by Atlassian JIRA (v7.6.3#76005)