Hi Fabian & Hequn,

Thank you for your responses. I am just responding now as I was out of office 
for the last few days

You mentioned that clear() is called when the time exceeds the window’s end 
timestamp. For my application I am using a GlobalWindow on a keyed stream -- 
would clear() get called at all in this case or should I be calling it manually?


Andrew

> On Oct 12, 2018, at 12:48 AM, Fabian Hueske <fhue...@gmail.com> wrote:
> 
> Hi Andrew,
> 
> The PURGE action of a window removes the window state (i.e., the collected 
> events or computed aggregate) but the window meta data including the Trigger 
> remain.
> The Trigger.close() method is called, when the winodw is completely (i.e., 
> all meta data) discarded. This happens, when the time (wallclock time for 
> processing time or watermark for event time windows) exceeds the window's end 
> timestamp.
> 
> Best, Fabian
> 
> Am Fr., 12. Okt. 2018 um 05:25 Uhr schrieb Hequn Cheng <chenghe...@gmail.com 
> <mailto:chenghe...@gmail.com>>:
> Hi Andrew,
> 
> Do you use CountWindow? You can switch to TimeWindow to have a test.
> I'm not quite familiar with window. I checked the code and found that clear() 
> is called only when timer is triggered, i.e, called at the end of time window.
> Hope this helps.
> 
> Best, Hequn
> 
> On Fri, Oct 12, 2018 at 6:23 AM Andrew Danks <a.da...@gmail.com 
> <mailto:a.da...@gmail.com>> wrote:
> Hello,
> 
> I see that the clear() function is implemented for various types of Triggers 
> in the Flink API. For example:
> https://github.com/apache/flink/blob/release-1.3/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java#L87
>  
> <https://github.com/apache/flink/blob/release-1.3/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/ContinuousProcessingTimeTrigger.java#L87>
> 
> I am working on a custom Trigger for my application and have implemented 
> clear() in a similar way.
> 
> However, having put a breakpoint in this function it doesn’t seem to get 
> called when I expect. The source code says that is called "when a window is 
> purged”[1] but when my Trigger emits a PURGE this function never seems to get 
> called. I am on Flink 1.3.
> 
> Hoping someone can shed more light on the purpose of clear() and how/when it 
> gets called
> 
> Thanks!
> Andrew
> 
> 
> [1] 
> https://github.com/apache/flink/blob/release-1.3/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java#L111
>  
> <https://github.com/apache/flink/blob/release-1.3/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/triggers/Trigger.java#L111>
> 

Reply via email to