Re: [Discussion] Query regarding Join and Windows

2016-07-01 Thread Aljoscha Krettek
Hi, yes, the window operator is stateful, which means that it will pick up where it left in case of a failure and restore. You're right about the graph, chained operators are shown as one box. Cheers, Aljoscha On Fri, 1 Jul 2016 at 04:52 Vinay Patil wrote: > Hi, > > Just watched the video on R

Re: [Discussion] Query regarding Join and Windows

2016-06-30 Thread Vinay Patil
Hi, Just watched the video on Robust Stream Processing . So when we say Window is a stateful operator , does it mean that even if the task manager doing the window operation fails, will it pick up from the state left earlier when it comes up ? (Have not read more on state for now) Also in one o

Re: [Discussion] Query regarding Join and Windows

2016-06-30 Thread Vinay Patil
Hi Aljoscha, Just wanted to check if it works with it. Anyways to solve the problem what we have thought of is to push heartbeat message to Kafka after certain interval, so that we get continuous stream always and that edge case will never occur, right ? One more question I have regarding the fai

Re: [Discussion] Query regarding Join and Windows

2016-06-30 Thread Aljoscha Krettek
Hi, I think the problem is that the DeltaFunction needs to have this signature: DeltaFunction, Tuple2>> because the Trigger will see elements from both input streams which are represented as a TaggedUnion that can contain an element from either side. May I ask why you want to use the DeltaTrigge

Re: [Discussion] Query regarding Join and Windows

2016-06-29 Thread Vinay Patil
Hi, Yes , now I am getting clear with the concepts here. One last thing I want to try before going for custom trigger, I want to try Delta Trigger but I am not able to get the syntax right , this is how I am trying it : TypeInformation> typeInfo = TypeInformation.of(new TypeHint>() { }); // sourc

Re: [Discussion] Query regarding Join and Windows

2016-06-29 Thread Aljoscha Krettek
Hi, you can use ingestion time if you don't care about the timestamps in your events, yes. If elements from the two streams happen to arrive at such times that they are not put into the same window then you won't get a match, correct. Regarding ingestion time and out-of-order events. I think this

Re: [Discussion] Query regarding Join and Windows

2016-06-29 Thread Vinay Patil
Hi, Ok. Inside the checkAndGetNextWatermark(lastElement, extractedTimestamp) method both these parameters are coming same (timestamp value) , I was expecting last element timestamp value in the 1st param when I extract it. Lets say I decide to use IngestionTime (since I am getting accurate result