[ 
https://issues.apache.org/jira/browse/FLINK-18646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17163083#comment-17163083
 ] 

Till Rohrmann commented on FLINK-18646:
---------------------------------------

[~TsReaper] do you know how the high number of GC calls happens? Looking at the 
code, we must have increased the {{sleeps}} counter to 9 and then after calling 
{{System.gc()}} once {{JavaGcCleanerWrapper.tryRunPendingCleaners()}} must 
return {{true}} for several calls which will keep the counter at 9 and calling 
{{System.gc()}} for every pending cleaner. This looks not like an intended 
behavior to me.

[~azagrebin] how long do we have to wait until we can expect that all segments 
have been detected by GC so that we can run the respective cleaners? What do 
you mean with keeping some buffer to compensate for GC? What do you mean with 
hitting the global limit?

> Managed memory released check can block RPC thread
> --------------------------------------------------
>
>                 Key: FLINK-18646
>                 URL: https://issues.apache.org/jira/browse/FLINK-18646
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Task
>    Affects Versions: 1.11.0
>            Reporter: Andrey Zagrebin
>            Assignee: Andrey Zagrebin
>            Priority: Critical
>             Fix For: 1.11.2
>
>         Attachments: log1.png, log2.png
>
>
> UnsafeMemoryBudget#verifyEmpty, called on slot freeing, needs time to wait on 
> GC of all allocated/released managed memory. If there are a lot of segments 
> to GC then it can take time to finish the check. If slot freeing happens in 
> RPC thread, the GC waiting can block it and TM risks to miss its heartbeat.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to