[ https://issues.apache.org/jira/browse/KAFKA-8924?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16933966#comment-16933966 ]
John Roesler commented on KAFKA-8924: ------------------------------------- Hey, all, I'll readily admit that the internal code is confusing and misleading. The behavior is also unfortunate, but I think it's basically unavoidable. I hope the documentation at least warns you of this effect. In 2.3, we've added an info log that prints out the grace period if you're using Suppress. If you can point out where we can add better documentation so that you would have seen this up front, please let us know. One thing we can clear up: the default value is _not_ -1, and it's not correct to say that the window will "never close". *The default is 24 hours*. We initialize the internal field to -1 because we need to distinguish between "no explicit grace period was set" and "an explicit grace period was set to some value (which could be 24 hours)". The logic in the `gracePeriod()` method, quoted above, then takes care of falling back to the retention time if no grace period was set. If no retention time was set as well, then the default retention time (24 hours) also becomes the grace period. Following from this, the behavior is not that "results are never emitted", just that "results are emitted after a full day". Not that the experience is any nicer, but it's an important distinction from a bug/no bug perspective. Regarding why the default grace period falls back to the retention time, rather than defaulting to 0... I would have preferred to default to 0 for suppress, but outside of the suppression context, a default grace period would mean that Streams doesn't handle out-of-order data at all by default, which would be a major step backwards. I've mulled this over many times, and I've just been doing it again... Maybe we can consider actually just requiring grace period to be explicitly set when you use Suppress.untilWindowCloses (throwing a runtime exception if it's not explicitly set). Then, we wouldn't need to pick a default at all. It seems hard to introduce a new exception in a minor release, but maybe in this case it would be ok. I guess we might be able to assume that practically nobody would want the default of 24 hours with Suppressed.untilWindowCloses, and therefore, anyone who is already using the feature would have set the grace period already, so almost no one would get a new exception upon upgrading. I'm not sure... what are your thoughts? > Default grace period (-1) of TimeWindows causes suppress to never emit events > ------------------------------------------------------------------------------ > > Key: KAFKA-8924 > URL: https://issues.apache.org/jira/browse/KAFKA-8924 > Project: Kafka > Issue Type: Bug > Components: streams > Affects Versions: 2.3.0 > Reporter: MichaĆ > Priority: Major > > 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)