[ 
https://issues.apache.org/jira/browse/KAFKA-8613?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16936335#comment-16936335
 ] 

Matthias J. Sax commented on KAFKA-8613:
----------------------------------------

I think the tricky part is expected default behavior for an aggregation itself 
and the difference in what might be better default behavior with and without 
suppress. Originally, we only had retention time and set it to 24h to allow 
handling out-of-order records out-of-the-box.

Thinking about it more, this previous decision might not have been the best 
one. If we would have set retention time to zero by default, and hence force 
people to opt-in for longer retention time and thus opt-in to handle 
out-of-order data, it might have been better (also consider the required 
storage to store 24h worth of data). Note that seems to be quite consistent 
with regard to windowed vs non-windowed aggregation, too. There is also not 
good justification to pick 24h as default value (why not 7days? why not 12h? or 
just 1h?).

>From an API point of view, it seem undesirable to throw a runtime exception 
>though. And it might also be undesired to have different behavior with and 
>without suppress...

Overall, I think we have two options:
 # set default grace period and default retention time to zero
 # make grace period a mandatory argument (enforce by the API, ie, not runtime 
exception)

I think that (1) has the big disadvantage that we advertise Kafka Streams as 
"can handle out-of-order data" but we force users to opt-in. For (2), the 
disadvantage is that we force people to think about it, ie, we "violate" the 
idea to keep the API as simple as possible.

Given the current confusion about suppress(), atm I tend to think that (1) is 
the slightly better option.

What we should not do, from my point of view:
 * throw runtime exceptions
 * have different default (windowed vs non-windowed and with/without suppress)

Thoughts?

> Set default grace period to 0
> -----------------------------
>
>                 Key: KAFKA-8613
>                 URL: https://issues.apache.org/jira/browse/KAFKA-8613
>             Project: Kafka
>          Issue Type: Improvement
>          Components: streams
>            Reporter: Bruno Cadonna
>            Priority: Blocker
>             Fix For: 3.0.0
>
>
> Currently, the grace period is set to retention time if the grace period is 
> not specified explicitly. The reason for setting the default grace period to 
> retention time was backward compatibility. Topologies that were implemented 
> before the introduction of the grace period, added late arriving records to a 
> window as long as the window existed, i.e., as long as its retention time was 
> not elapsed.  
> This unintuitive default grace period has already caused confusion among 
> users.
> For the next major release, we should set the default grace period to 
> {{Duration.ZERO}}.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to