I think you are all right. I have I checked the source code of WatermarkAssignerOperator, and I have found the WatermarkGenerator passed in WatermarkAssignerOperator is the interface WatermarkGenerator. And BoundedOutOfOrderWatermarkGenerator is the only implementation class of WatermarkGenerator. By the way , the interval is based processing time .
Benchao Li <libenc...@gmail.com> 于2020年4月17日周五 下午5:06写道: > WatermarkAssignerOperator is an inner mechanism for generating watermarks. > > The "Bounded Out of Orderness" is just one kind of the watermark > expressions, which > is most commonly used. > > The main logic of WatermarkAssignerOperator is: > - keep currentWatermark and lastWatermark > - when each element comes in > - get watermark from this element, using the *watermark expression* > - if the watermark > currentWatermark, then currentWatermark is updated > - if currentWatermark - lastWatermark > watermarkInterval > - emit watermark to downstream, and update lastWatermark > > lec ssmi <shicheng31...@gmail.com> 于2020年4月17日周五 下午4:50写道: > >> Maybe you are all right. I was more confused . >> As the cwiki said, flink could use BoundedOutOfOrderTimestamps , >> [image: image.png] >> >> but I have heard about WatermarkAssignerOperator from Blink developers. >> >> Benchao Li <libenc...@gmail.com> 于2020年4月17日周五 下午4:33写道: >> >>> Hi lec ssmi, >>> >>> It's a good question. In blink planner, we use code gen to handle >>> watermark expression. >>> And in `WatermarkAssignerOperator` we calculate current watermark when >>> each element comes in. >>> If the watermark - lastEmitedWatermark > watermark interval, we will >>> emit the new watermark. >>> >>> So it's neither `PeriodicWatermark` nor `PunctuatedWatermark`. >>> >>> lec ssmi <shicheng31...@gmail.com> 于2020年4月17日周五 下午3:12写道: >>> >>>> Hi: >>>> In sql API , the declaration of watermark is realized by ddl >>>> statement . But which way is it implemented? >>>> * PeriodicWatermark * or *PunctuatedWatermark*? >>>> There seems to be no explanation on the official website. >>>> >>>> Thanks. >>>> >>> >>> >>> -- >>> >>> Benchao Li >>> School of Electronics Engineering and Computer Science, Peking University >>> Tel:+86-15650713730 >>> Email: libenc...@gmail.com; libenc...@pku.edu.cn >>> >>> > > -- > > Benchao Li > School of Electronics Engineering and Computer Science, Peking University > Tel:+86-15650713730 > Email: libenc...@gmail.com; libenc...@pku.edu.cn > >