Re: Kafka Streams - issues with windowing and suppress

2020-04-21 Thread Liam Clarke-Hutchinson
Hi John, Yep, I saw there was a few issues filed around default grace periods, and I have a few ideas about sensible defaults and possible APIs. I'll sign up for a Jira account in the next few days to join the discussion :) Kind regards, Liam Clarke-Hutchinson On Tue, Apr 21, 2020 at 9:09 AM

Re: Kafka Streams - issues with windowing and suppress

2020-04-20 Thread John Roesler
Yes, thanks, Liam! By the way, There's actually already a ticket to try and improve the API, and the discussed solution is basically the same thing I said had never occurred to me before, so I'm not sure what to say about that... https://issues.apache.org/jira/browse/KAFKA-8924 The ticket

Re: Kafka Streams - issues with windowing and suppress

2020-04-20 Thread Matthias J. Sax
Thanks for the PR! On 4/19/20 10:04 PM, Liam Clarke-Hutchinson wrote: > PR submitted :) https://github.com/apache/kafka/pull/8520 > > On Mon, Apr 20, 2020 at 2:34 PM John Roesler wrote: > >> Hi Liam, >> >> That sounds like a good idea to me. In fact, I’d go so far as to say we >> should just

Re: Kafka Streams - issues with windowing and suppress

2020-04-19 Thread Liam Clarke-Hutchinson
PR submitted :) https://github.com/apache/kafka/pull/8520 On Mon, Apr 20, 2020 at 2:34 PM John Roesler wrote: > Hi Liam, > > That sounds like a good idea to me. In fact, I’d go so far as to say we > should just change the existing example to include a grace period, and not > bother with an

Re: Kafka Streams - issues with windowing and suppress

2020-04-19 Thread John Roesler
Hi Liam, That sounds like a good idea to me. In fact, I’d go so far as to say we should just change the existing example to include a grace period, and not bother with an extra example. That would put it front and center. A PR would be greatly appreciated! Thanks for the offer! Thanks, John

Re: Kafka Streams - issues with windowing and suppress

2020-04-19 Thread Liam Clarke
Hi Matthias, I think as an interim measure, if the windowing samples in the docs showed an additional example where the grace period was set (with perhaps a comment about the current default grace period, and planned future changes?) it would make it sufficiently visible - happy to submit a PR

Re: Kafka Streams - issues with windowing and suppress

2020-04-19 Thread Matthias J. Sax
I would prefer to not make the grace-period a mandatory argument and keep the API as-is. I understand the issue of backward compatibility, but I would still argue that we should just change the default grace period to 0 in the 3.0 release. It's a major release and thus it seems to be fine. To

Re: Kafka Streams - issues with windowing and suppress

2020-04-19 Thread John Roesler
Oh, man, that’s a good idea. I can propose to deprecate (not remove) the existing ‘of’ factory method and add one with a mandatory grace period. Not sure why I didn’t think of that before. Probably too caught up in looking for something “smart”. Thanks! John On Sun, Apr 19, 2020, at 02:27,

Re: Kafka Streams - issues with windowing and suppress

2020-04-19 Thread Liam Clarke
Hi John, I can't really think of a way to make it more obvious without breaking backwards compatibility - e.g., obvious easy fix is that grace period is a mandatory arg to TimeWindows, but that would definitely break compatibility. Cheers, Liam Clarke-Hutchinson On Thu, Apr 16, 2020 at 1:59 AM

Re: Kafka Streams - issues with windowing and suppress

2020-04-15 Thread John Roesler
Boom, you got it, Liam! Nice debugging work. This is a pretty big bummer, but I had to do it that way for compatibility. I added a log message to try and help reduce the risk, but it’s still kind of a trap. I’d like to do a KIP at some point to consider changing the default grace period,

Re: Kafka Streams - issues with windowing and suppress

2020-04-14 Thread Liam Clarke
And the answer is to change .windowedBy(TimeWindows.of(Duration.ofMillis(5000))) and specify the grace period: windowedBy(TimeWindows.of(Duration.ofMillis(5000)).grace(Duration.ofMillis(100))) On Wed, Apr 15, 2020 at 4:34 PM Liam Clarke wrote: > Okay, doing some debugging it looks like I'm

Re: Kafka Streams - issues with windowing and suppress

2020-04-14 Thread Liam Clarke
Okay, doing some debugging it looks like I'm seeing this behaviour because it's picking up a grace duration of 86,395,000 ms in KTableImpl.buildSuppress, which would happen to be 5000 millis (my window size) off 24 hours, so I've got some clues! On Wed, Apr 15, 2020 at 3:43 PM Liam Clarke

Kafka Streams - issues with windowing and suppress

2020-04-14 Thread Liam Clarke
Hi all, I have a case where I want to consume from a topic, count the number of certain ids in a given time period X, and emit a new record to a different topic after that same time period X has elapsed containing the aggregated value. I'm using suppress with Suppressed.untilWindowCloses, but