Hi Ankur,

Thanks for the KIP! A built-in approach for TTLs in state store is an
important extension of Kafka Streams. Indeed, it does not seem ideal
that people must implement this manually by writing clean-up
tombstones from a punctuation, particularly because it requires
scanning the entire state store and will not scale to larger state
stores.

I like how your KIP makes it extremely simple to set up a TTL by just
calling withTtl on the state store. However, the approach to expiring
records is still based on periodic scans of the entire state store.
Additionally, the punctuation interval appears to be hardcoded to one
minute.

I was wondering if you have explored alternative implementations to
expire keys in a way that scales to larger stores as well? One idea is
to use RocksDB's "best effort" Compaction Filter and retention on the
changelog topic to reclaim space based on the TTL. Then, ensure a
consistent view of the local state store and a state store restored
from the changelog topic by precisely enforcing the TTL when the state
store is accessed. Another idea is to change the underlying RocksDB
representation to perform periodic expiration more efficiently, for
example, by having an index by timestamp and expiring keys using a
prefix scan on that index.

These are vague ideas that require detailed validation. But I am not
sure that we should introduce a TTL expiration mechanism that doesn't
scale to larger state stores before exploring these options.

WDYT?

Cheers,
Lucas

On Tue, Oct 7, 2025 at 7:26 AM Ankur Sinha <[email protected]> wrote:
>
> Hello,
>
> I’d like to start the discussion 🙂  for KIP-1225: Add Optional TTL Support to 
> Kafka Streams State 
> Stores<https://cwiki.apache.org/confluence/display/KAFKA/KIP-1225%3A+Add+Optional+TTL+Support+to+Kafka+Streams+State+Stores>
> JIRA: 
> https://issues.apache.org/jira/browse/KAFKA-19759<https://issues.apache.org/jira/browse/KAFKA-19759>
> KIP Wiki: 
> https://cwiki.apache.org/confluence/display/KAFKA/KIP-1225%3A+Add+Optional+TTL+Support+to+Kafka+Streams+State+Stores
>
> Regards,
> Ankur Sinha
>
> ________________________________
> From: Matthias J. Sax <[email protected]>
> Sent: Tuesday, October 7, 2025 1:04 AM
> To: [email protected] <[email protected]>
> Subject: [EXT] Re: [DISCUSS] KIP-19759: Add built-in TTL (Time-to-Live) 
> support for Kafka Streams State Stores
>
> Warning External sender Do not click on any links or open any attachments 
> unless you trust the sender and know the content is safe.
>
> If you want to start a KIP, you need to write one :) A Jira ticket is
> not a KIP
>
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals<https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals>
>
>
> -Matthias
>
> This email was screened for spam and malicious content but exercise caution 
> anyway.
>
>
>
> On 10/6/25 10:26 AM, Ankur Sinha wrote:
> > Hi all,
> >
> > I’d like to start the discussion for KAFKA-19759: Add built-in TTL 
> > (Time-to-Live) support for Kafka Streams State Stores.
> >
> > JIRA: 
> > https://issues.apache.org/jira/browse/KAFKA-19759<https://issues.apache.org/jira/browse/KAFKA-19759>
> > KIP Wiki: (will be created shortly)
> >
> > Streams users often need per-key TTL functionality (e.g., for cache-like 
> > state, per-key deduplication, or automatic cleanup).
> > Currently, this requires manual punctuators and tombstones.
> >
> > This KIP proposes adding a built-in `withTtl(Duration ttl)` option to state 
> > stores,
> > which would handle key expiration and changelog consistency automatically.
> >
> > Please see the JIRA for full details and general analysis.
> > Looking forward for a discussion and talk.
> >
> > Best regards,
> > Ankur Sinha
> >

Reply via email to