Kafka Streams does not change topic configs if a topic exists already.
Thus, there should be no impact in existing application if they are
upgraded.

However, I share the backward compatibility concern in general.
Furthermore, for windowing, retention period must be at least window
size IIRC, thus, if we change it, we should set the default accordingly.

I still have my doubts, if a short retention time like this would be a
good out-of-the-box experience, because we might loose out-of-order data
that miss the window-end time...

        
-Matthias

On 1/3/19 9:05 PM, Boyang Chen wrote:
> Thanks for the proposal Jingguo. Guozhang when you mention to "change the
> default value of retention to 0 consistently", will we introduce any backward 
> incompatible issue since
> the retention cutoff might change on the changelog topics that are unexpected.
> 
> Boyang
> 
> ________________________________
> From: Guozhang Wang <wangg...@gmail.com>
> Sent: Friday, January 4, 2019 3:16 AM
> To: dev
> Subject: Re: Suggestion to make 0 grace period the default for suppress
> 
> Thanks for reporting this Jingguo, personally I'd favor we change the
> default value of retention to 0 consistently, regardless of whether
> suppress is used, if we would ever consider changing it.
> 
> Since it is a public API change (i.e. changing the default value), it
> should be discussed and voted via a KIP:
> https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Improvement+Proposals
> 
> Are you interested in creating one and drive the discussion?
> 
> 
> Guozhang
> 
> On Sun, Dec 30, 2018 at 10:34 PM jingguo yao <yaojing...@gmail.com> wrote:
> 
>> [1] has the following code to demonstrate the usage of suppress method.
>>
>> KGroupedStream<UserId, Event> grouped = ...;
>> grouped
>>   .windowedBy(TimeWindows.of(Duration.ofHours(1)).grace(ofMinutes(10)))
>>   .count()
>>   .suppress(Suppressed.untilWindowCloses(unbounded()))
>>   .filter((windowedUserId, count) -> count < 3)
>>   .toStream()
>>   .foreach((windowedUserId, count) ->
>> sendAlert(windowedUserId.window(), windowedUserId.key(), count));
>>
>>
>> If I remove the grace method invocation, I will have a one day
>> retention period. The following code in
>> org.apache.kafka.streams.kstream.TimeWindows causes this hehaviour:
>>
>> @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 grace != null ? grace.toMillis() : maintainMs() - size();
>> }
>>
>> I think that it is better to use 0 grace period if
>> "suppress(Suppressed.untilWindowCloses(unbounded()))" exists. With the
>> suppress method invocation, people are expecting to see the final
>> window result when the window closes instead of wait to see the result
>> after the one-day period. Even if we have some reasons similar to ones
>> mentioned in the code comment, it is better to mention this hehaviour
>> somewhere in Kafka streams documentation.
>>
>>
>> [1]
>> http://kafka.apache.org/21/documentation/streams/developer-guide/dsl-api.html#window-final-resu
>>
>>
>> --
>> Jingguo
>>
> 
> 
> --
> -- Guozhang
> 

Attachment: signature.asc
Description: OpenPGP digital signature

Reply via email to