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

John Roesler commented on KAFKA-8924:
-------------------------------------

Thanks for the comment, [~cadonna] ,

I agree that it's not as elegant-looking this way, but the point is that if we 
make it two separate methods, then we _have to_ select a default. I'm leaning 
toward the viewpoint that there is no sensible default grace period, therefore 
it's got to be a required parameter, which means that it must be present in the 
"root" factory method arguments.

I can see now why other stream processing systems attempt to adaptively "learn" 
the grace period by observing the input stream's lateness, but I'm still on the 
fence about whether that's really a good "default" either.

> Default grace period (-1) of TimeWindows causes suppress to emit events after 
> 24h
> ---------------------------------------------------------------------------------
>
>                 Key: KAFKA-8924
>                 URL: https://issues.apache.org/jira/browse/KAFKA-8924
>             Project: Kafka
>          Issue Type: Improvement
>          Components: streams
>    Affects Versions: 2.3.0
>            Reporter: Michał
>            Assignee: Michał
>            Priority: Major
>              Labels: needs-kip
>
> h2. Problem 
> The default creation of TimeWindows, like
> {code}
> TimeWindows.of(ofMillis(xxx))
> {code}
> calls an internal constructor
> {code}
> return new TimeWindows(sizeMs, sizeMs, -1, DEFAULT_RETENTION_MS);
> {code}
> And the *-1* parameter is the default grace period which I think is here for 
> backward compatibility
> {code}
> @SuppressWarnings("deprecation") // continuing to support 
> Windows#maintainMs/segmentInterval in fallback mode
> @Override
> public long gracePeriodMs() {
>     // NOTE: in the future, when we remove maintainMs,
>     // we should default the grace period to 24h to maintain the default 
> behavior,
>     // or we can default to (24h - size) if you want to be super accurate.
>     return graceMs != -1 ? graceMs : maintainMs() - size();
> }
> {code}
> The problem is that if you use a TimeWindows with gracePeriod of *-1* 
> together with suppress *untilWindowCloses*, it never emits an event.
> You can check the Suppress tests 
> (SuppressScenarioTest.shouldSupportFinalResultsForTimeWindows), where 
> [~vvcephei] was (maybe) aware of that and all the scenarios specify the 
> gracePeriod.
> I will add a test without it on my branch and it will fail.
> The test: 
> https://github.com/atais/kafka/commit/221a04dc40d636ffe93a0ad95dfc6bcad653f4db
>  
> h2. Now what can be done
> One easy fix would be to change the default value to 0, which works fine for 
> me in my project, however, I am not aware of the impact it would have done 
> due to the changes in the *gracePeriodMs* method mentioned before.



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

Reply via email to