[ 
https://issues.apache.org/jira/browse/KAFKA-7072?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Guozhang Wang resolved KAFKA-7072.
----------------------------------
    Resolution: Fixed

> Kafka Streams may drop rocksb window segments before they expire
> ----------------------------------------------------------------
>
>                 Key: KAFKA-7072
>                 URL: https://issues.apache.org/jira/browse/KAFKA-7072
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 0.11.0.0, 0.11.0.1, 1.0.0, 0.11.0.2, 1.1.0, 2.0.0, 1.0.1
>            Reporter: John Roesler
>            Assignee: John Roesler
>            Priority: Minor
>             Fix For: 2.1.0
>
>
> The current implementation of Segments used by Rocks Session and Time window 
> stores is in conflict with our current timestamp management model.
> The current segmentation approach allows configuration of a fixed number of 
> segments (let's say *4*) and a fixed retention time. We essentially divide up 
> the retention time into the available number of segments:
> {quote}{{<---------|-----------------------------|}}
>  {{   expiration date                 right now}}
>  {{          -------retention time--------/}}
>  {{          |  seg 0  |  seg 1  |  seg 2  |  seg 3  |}}
> {quote}
> Note that we keep one extra segment so that we can record new events, while 
> some events in seg 0 are actually expired (but we only drop whole segments, 
> so they just get to hang around.
> {quote}{{<-------------|-----------------------------|}}
>  {{       expiration date                 right now}}
>  {{              -------retention time--------/}}
>  {{          |  seg 0  |  seg 1  |  seg 2  |  seg 3  |}}
> {quote}
> When it's time to provision segment 4, we know that segment 0 is completely 
> expired, so we drop it:
> {quote}{{<-------------------|-----------------------------|}}
>  {{             expiration date                 right now}}
>  {{                    -------retention time--------/}}
>  {{                    |  seg 1  |  seg 2  |  seg 3  |  seg 4  |}}
> {quote}
>  
> However, the current timestamp management model allows for records from the 
> future. Namely, because we define stream time as the minimum buffered 
> timestamp (nondecreasing), we can have a buffer like this: [ 5, 2, 6 ], and 
> our stream time will be 2, but we'll handle a record with timestamp 5 next. 
> referring to the example, this means we could wind up having to provision 
> segment 4 before segment 0 expires!
> Let's say "f" is our future event:
> {quote}{{<-------------------|-----------------------------|----f}}
>  {{             expiration date                 right now}}
>  {{                    -------retention time--------/}}
>  {{                    |  seg 1  |  seg 2  |  seg 3  |  seg 4  |}}
> {quote}
> Should we drop segment 0 prematurely? Or should we crash and refuse to 
> process "f"?
> Today, we do the former, and this is probably the better choice. If we refuse 
> to process "f", then we cannot make progress ever again.
> Dropping segment 0 prematurely is a bummer, but users could also set the 
> retention time high enough that they don't think they'll actually get any 
> events late enough to need segment 0. Worst case, since we can have many 
> future events without advancing stream time, sparse enough to each require 
> their own segment, which would eat deeply into the retention time, dropping 
> many segments that should be live.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to