Hi Yun, Thanks for the proposal.
It appears that the issues mentioned in the motivation section are all related to using Windows on the DataStream API, where the user's code typically does not have anything to do with Timer. I am wondering if the following alternative solution might be more user-friendly. Here is a sketch of the alternative solution: 1) Add the method *abstract* *TriggerResult onEndOfStream(W window, TriggerContext ctx)* to the existing abstract class Trigger. This method allows the Trigger subclass to additionally handle the end-of-stream signal. 2) Add a Trigger subclass *EndOfStreamTrigger*. This trigger takes a nested trigger and expectedTriggerResult as its constructor parameters and can apply extra onEndOfStream(...) logic on the existing trigger classes. 3) For users who want to fire pending windows on end-of-stream, users can do something like this: *datastream.keyBy(...).window(...).trigger(EndOfStreamTrigger.of(existingTrigger, TriggerResult.FIRE_AND_PURGE)).* IMO, the main benefit of this alternative solution is that it is more consistent with the existing Windows API. Users who are concerned with firing windows on end-of-stream won't need to additionally understand/handle timer. What do you think? I might have missed use-cases for this FLIP which do not involve windows. If so, could you help explain the use-case in this FLIP? Cheers, Dong On Wed, Nov 2, 2022 at 6:20 PM Yun Gao <yungao...@aliyun.com.invalid> wrote: > Hi everyone, > I would like to open a discussion[1] on how to > properly handle the processing timers on job > termination. > Currently all the processing timers would be > ignored on job termination. This behavior is > not suitable for some cases like WindowOperator. > Thus we'd like to provide more options for how > to deal with the pending times on job termination, > and provide correct semantics on bounded stream > for these scenarios. The FLIP is based on the previous > discussion with Piotr and Divye in [2]. > [1] > https://cwiki.apache.org/confluence/display/FLINK/FLIP-269%3A+Properly+Handling+the+Processing+Timers+on+Job+Termination > < > https://cwiki.apache.org/confluence/display/FLINK/FLIP-269%3A+Properly+Handling+the+Processing+Timers+on+Job+Termination > > > [2] https://issues.apache.org/jira/browse/FLINK-18647 < > https://issues.apache.org/jira/browse/FLINK-18647 > >