Re: Whey does the window final result is not emitted after the window has elapsed?

2019-01-04 Thread jingguo yao
Guozhang: Thanks for your kind help. Guozhang Wang 于2019年1月5日周六 上午3:28写道: > > Thanks for the detailed description. > > 1) Yes the stream time is advancable by any records. > 2) Given your description, another way to work around the situation is to > let your class send a final record with

Re: Whey does the window final result is not emitted after the window has elapsed?

2019-01-04 Thread Guozhang Wang
Thanks for the detailed description. 1) Yes the stream time is advancable by any records. 2) Given your description, another way to work around the situation is to let your class send a final record with timestamp set as the class-end-time plus a small delta (think of it as a sentinel "tick"

Re: Whey does the window final result is not emitted after the window has elapsed?

2019-01-03 Thread jingguo yao
Guozhang: Yes, my case is a real production scenario. I am monitoring on-line live-broadcast classes. I need to have a summary of each 5-minute period for one class. Each class has a classroom Id. I report class activity data to a Kafka topic. Classroom id is used to partition these data. Here

Re: Whey does the window final result is not emitted after the window has elapsed?

2019-01-03 Thread Guozhang Wang
Hello Jingguo, Is this case (i.e. you only have data over 57 minutes, and no new data afterwards) a real production scenario? In stream processing we usually expect the input data stream in continuously, and I'm curious to learn your use case better and why it would not have further data after a

Re: Whey does the window final result is not emitted after the window has elapsed?

2019-01-03 Thread jingguo yao
Hi, Matthias I am doing a 5-minute tumbling window analysis over a 57-minute data flow. And I want only one final result per window. So I need suppress. The 57-minute period can be divided into about 12 windows. The results of the first 11 windows can be delivered downstream. But the final result

Re: Whey does the window final result is not emitted after the window has elapsed?

2019-01-02 Thread Matthias J. Sax
> After some time, the window closes. This is not correct. Windows are based on event-time, and because no new input record is processed, the window is not closed. That is the reason why you don't get any output. Only a new input record can advance "stream time" and close the window. In

Re: Whey does the window final result is not emitted after the window has elapsed?

2018-12-30 Thread jingguo yao
Sorry for my typo in the mail. "Whey" should be "Why" in "Whey does the window final result is not emitted after the window has elapsed?" I have browsed the Kafka source code and found the cause of the mentioned behaviour.

Whey does the window final result is not emitted after the window has elapsed?

2018-12-30 Thread jingguo yao
I followed [1] to code a simple example to try suppress operator. Here is the simple code: final Serde stringSerde = Serdes.String(); final StreamsBuilder builder = new StreamsBuilder(); builder.stream("TextLinesTopic", Consumed.with(Serdes.String(), Serdes.String())) .flatMapValues(value ->