[ https://issues.apache.org/jira/browse/FLINK-18646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17163285#comment-17163285 ]
Andrey Zagrebin edited comment on FLINK-18646 at 7/23/20, 7:33 AM: ------------------------------------------------------------------- {quote}do you know how the high number of GC calls happens? {quote} >From the offline discussion: 400 actual GCs, the number of System.gc() calls >is indeed high for each while iteration {quote}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. {quote} >From the offline discussion: this is true, it was overlooked. That is why it >makes sense to call GC only if 'tryRunPendingCleaners' is false after sleep >and memory still cannot be allocated. {quote}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? {quote} JVM waits for 9 sleeps (0,5) for direct memory but it is generally used for large global limit and allocations which are small relative to the limit which requires to GC only so much as asked. {quote}What do you mean with keeping some buffer to compensate for GC? {quote} If 120 Mb is the actual memory that can be used then make the interface limit of 100 Mb. Keep 20 Mb to allocate more actual memory than 100Mb if some memory is not GC'ed yet. {quote}What do you mean with hitting the global limit? {quote} When we allocated all available actual memory, some of which is released but not GC'ed yet, and we try to allocate more assuming that we released some and it should be available. was (Author: azagrebin): {quote}do you know how the high number of GC calls happens? {quote} >From the offline discussion: 400 actual GCs, the number of System.gc() calls >is indeed high for each while iteration {quote}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. {quote} >From the offline discussion: this is true, it was overlooked. That is why it >makes sense to continue calling GC only if 'tryRunPendingCleaners' is false >after sleep and memory still cannot be allocated. {quote}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? {quote} JVM waits for 9 sleeps (0,5) for direct memory but it is generally used for large global limit and allocations which are small relative to the limit which requires to GC only so much as asked. {quote}What do you mean with keeping some buffer to compensate for GC? {quote} If 120 Mb is the actual memory that can be used then make the interface limit of 100 Mb. Keep 20 Mb to allocate more actual memory than 100Mb if some memory is not GC'ed yet. {quote}What do you mean with hitting the global limit? {quote} When we allocated all available actual memory, some of which is released but not GC'ed yet, and we try to allocate more assuming that we released some and it should be available. > 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)