[
https://issues.apache.org/jira/browse/FLINK-18646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17162693#comment-17162693
]
Andrey Zagrebin commented on FLINK-18646:
-----------------------------------------
I agree we should not trigger GC if the cleaner queue processing is active. If
we cannot still allocate memory and there are no ready cleaners detected by GC,
we have to re-trigger GC or fail after sleeping and timeout.
We can increase timeout for slot closing verification (off-loaded to another
thread) because we have to wait for GC of all segments to make sure there is no
leak. Normal reservation should probably have smaller timeout to fail faster in
case of leaks, especially if we do not use it in a way similar to verification:
allocate all, release and then re-allocate all again.
All-in-all, the timeouts have to be tuned. If we still find ourselves often
hitting the global limit, the global managed memory limit (not per slot) could
also compensate for GC and/or we should not allocate all available memory and
keep some buffer to compensate for GC which is usually the case for direct
memory.
> 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)