Hi,

There should be no window or trigger object kept per window.

Aljoscha

> On 15. Apr 2019, at 10:22, Fabian Hueske <fhue...@gmail.com> wrote:
> 
> Hi,
> 
> Aljoscha know the implementation best (since he implemented it). 
> 
> From my understanding (Aljoscha please correct me if I'm wrong), all Flink 
> managed state is removed (given that user-defined state is correctly cleaned 
> up).
> However, for each key, a window and a trigger object might be kept (this is 
> the part I'm not sure about). 
> This might cause memory issues if the keyspace is very large and "moving" 
> (keys which are only used for a short period of time, e.g., session keys).
> Eventually, a TM would fail causing job recovery. During recovery, only the 
> Flink managed state is restored and the TM would have free memory again.
> 
> Best, Fabian
> 
> 
> Am Fr., 12. Apr. 2019 um 19:58 Uhr schrieb M Singh <mans2si...@yahoo.com 
> <mailto:mans2si...@yahoo.com>>:
> Hi Fabian/Guowei:  
> 
> Thanks for your pointers.   
> 
> Fabian, as you pointed out, global window is never completely removed since 
> it's end time is Long.MAX_VALUE, and that is my concern.  So, is there any 
> other way of clean up the now purged global windows ?
> 
> Thanks again.
> 
> 
> 
> On Thursday, April 11, 2019, 4:16:24 AM EDT, Fabian Hueske <fhue...@gmail.com 
> <mailto:fhue...@gmail.com>> wrote:
> 
> 
> Hi,
> 
> As far as I know, a window is only completely removed when time (event or 
> processing time, depending on the window type) passes the window's end 
> timestamp.
> Since, GlobalWindow's end timestamp is Long.MAX_VALUE, it is never completely 
> removed.
> I'm not 100% sure what state is kept around. It might not be keyed state but 
> just objects on the heap but not absolutely sure.
> 
> Aljoscha (in CC) should know the details here.
> 
> Best, Fabian
> 
> Am Do., 11. Apr. 2019 um 08:07 Uhr schrieb Guowei Ma <guowei....@gmail.com 
> <mailto:guowei....@gmail.com>>:
> Hi,
> I think you could return a proper TriggerResult, which defines how to deal 
> with the window elements after computing a window in your trigger 
> implementation. You could find the detail information from the doc[1].
> 
> 1. 
> https://ci.apache.org/projects/flink/flink-docs-release-1.8/dev/stream/operators/windows.html#fire-and-purge
>  
> <https://ci.apache.org/projects/flink/flink-docs-release-1.8/dev/stream/operators/windows.html#fire-and-purge>
> Best,
> Guowei
> 
> 
> M Singh <mans2si...@yahoo.com <mailto:mans2si...@yahoo.com>> 于2019年4月11日周四 
> 上午1:42写道:
> Hi:
> 
> I have a use case where I need to create a global window where I need to wait 
> for unknown time for certain events for a particular key.  I understand that 
> I can create a global window and use a custom trigger to initiate the 
> function computation.  But I am not sure how to destroy the window after the 
> triggering conditions is satisfied and the the events are purged.
> 
> If there is any better way of dealing with this situation, please let me know.
> 
> Thanks
> 
> Mans

Reply via email to