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

Caizhi Weng edited comment on FLINK-18646 at 7/21/20, 8:12 AM:
---------------------------------------------------------------

*_the reserve operation might want to reserve just a bit and needs to wait for 
a couple of cleaners_*

I agree with this.

*_System.gc(), which is almost noop hint_*

For this I have some doubts. {{System.gc()}} is indeed just a hint, but we 
cannot make sure when the GC will happen after this method is called. It is 
possible that GC happens immediately after the call. I think we should only 
call {{System.gc()}} when {{tryRunPendingCleaners}} returns false, otherwise 
there are still pending cleaners and there is no need to hint for another GC.

I've added some logs before calling {{System.gc()}} in the infinite loop, and 
there are obvious gaps which are caused by full GCs.

!log1.png!

!log2.png!


was (Author: tsreaper):
_== the reserve operation might want to reserve just a bit and needs to wait 
for a couple of cleaners ==_

I agree with this.

_== System.gc(), which is almost noop hint ==_

For this I have some doubts. {{System.gc()}} is indeed just a hint, but we 
cannot make sure when the GC will happen after this method is called. It is 
possible that GC happens immediately after the call. I think we should only 
call {{System.gc()}} when {{tryRunPendingCleaners}} returns false, otherwise 
there are still pending cleaners and there is no need to hint for another GC.

I've added some logs before calling {{System.gc()}} in the infinite loop, and 
there are obvious gaps which are caused by full GCs.

!log1.png!

!log2.png!

> 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
>            Priority: Major
>         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