I was not able to edit the wiki page for the KIP. When you have a moment, please could you grant me permissions?
https://cwiki.apache.org/confluence/display/~iekpo On Wed, Apr 28, 2021 at 4:17 PM Israel Ekpo <israele...@gmail.com> wrote: > > Hi Everyone, > > I noticed that the method signature in the KIP had the incorrect return > value. > > > https://cwiki.apache.org/confluence/display/KAFKA/KIP-633:+Drop+24+hour+default+of+grace+period+in+Streams > > Should I go ahead and update this in the KIP? I just noticed the error > during my implementation > > For the class org.apache.kafka.streams.kstream.SessionWindows > > public static *JoinWindows *ofInactivityGapAndGrace(final Duration > inactivityGap, final Duration afterWindowEnd); > > Should have been: > > public static *SessionWindows *ofInactivityGapAndGrace(final Duration > inactivityGap, final Duration afterWindowEnd); > > I will update the KIP to reflect what I think it needs to be. Please let > me know your thoughts when you see this. > > Thanks > > On Thu, Apr 8, 2021 at 5:42 PM Matthias J. Sax <mj...@apache.org> wrote: > >> Thanks! >> >> On 4/8/21 2:06 PM, Sophie Blee-Goldman wrote: >> > 1) Since the new APIs (eg ofSizeWithNoGrace and ofSizeAndGrace) are the >> > only static constructors >> > after this change, there seems to be no reason to keep the .grace >> around -- >> > you've already specified >> > it with your choice of the static constructor. >> > >> > 2) Ack, updated the KIP >> > >> > 3) Ack, fixed >> > >> > On Tue, Apr 6, 2021 at 7:03 PM Matthias J. Sax <mj...@apache.org> >> wrote: >> > >> >> Thanks for the KIP Sophie. It make total sense to get rid of default >> >> grace period of 24h. >> >> >> >> >> >> Some questions/comments: >> >> >> >> (1) Is there any particular reason why we want to remove >> >> `grace(Duration)` method? >> >> >> >> >> >> (2) About `SlidingWindows#withTimeDifferenceAndGrace` -- personally I >> >> think it's worth to clean it up right now -- given that sliding windows >> >> are rather new the "splash radius" should be small. >> >> >> >> >> >> >> >> >> >> (3) Some nits on wording: >> >> >> >>> This config determines how long after a window closes any new data >> will >> >> still be processed >> >> >> >> Should be "after a window ends" -- a window is closed after grace >> period >> >> passed. >> >> >> >> >> >>> one which indicates to use no grace period and not handle out-of-order >> >> data >> >> >> >> Seems strictly not correct -- if there is a window from 0 to 100 and >> you >> >> get record with ts 99,98,97,...,0 all but the first of those records >> are >> >> out-of-order but they are still processed even with a grace period of >> zero. >> >> >> >> Maybe better: "one which indicate to use no grace period and close the >> >> window immediately when the window ends." >> >> >> >> >> >>> and make a conscious decision to skip the grace period and drop >> >> out-of-order records, >> >> >> >> Maybe better: "and make a conscious decision to skip the grace period >> >> and close a window immediately" >> >> >> >> >> >> >> >> -Matthias >> >> >> >> >> >> >> >> >> >> On 3/31/21 5:02 PM, Guozhang Wang wrote: >> >>> Hello Sophie, >> >>> >> >>> I agree that the old 24-hour grace period should be updated, and I >> also >> >>> think now it is a better idea to make the grace period "mandatory" >> from >> >> the >> >>> API names since it is a very important concept and hence worth >> >> emphasizing >> >>> to users up front. >> >>> >> >>> Guozhang >> >>> >> >>> On Wed, Mar 31, 2021 at 1:58 PM John Roesler <vvcep...@apache.org> >> >> wrote: >> >>> >> >>>> Thanks for bringing this up, Sophie! >> >>>> >> >>>> This has indeed been a pain point for a lot of people. >> >>>> >> >>>> It's a really thorny issue with no obvious "right" solution. >> >>>> I think your proposal is a good one. >> >>>> >> >>>> Thanks, >> >>>> -John >> >>>> >> >>>> On Wed, 2021-03-31 at 13:28 -0700, Sophie Blee-Goldman >> >>>> wrote: >> >>>>> Hey all, >> >>>>> >> >>>>> It's finally time to reconsider the default grace period in Kafka >> >>>> Streams, >> >>>>> and hopefully save a lot of suppression users from the pain of >> figuring >> >>>> out >> >>>>> why their results don't show up until 24 hours later. Please check >> out >> >>>> the >> >>>>> proposal and let me know what you think. >> >>>>> >> >>>>> KIP: >> >>>>> >> >>>> >> >> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-633%3A+Drop+24+hour+default+of+grace+period+in+Streams >> >>>>> < >> >>>> >> >> >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-633%3A+Drop+24hr+default+grace+period >> >>>>> >> >>>>> >> >>>>> JIRA: https://issues.apache.org/jira/browse/KAFKA-8613 >> >>>>> >> >>>>> Cheers, >> >>>>> Sophie >> >>>> >> >>>> >> >>>> >> >>> >> >> >> > >> >