[
https://issues.apache.org/jira/browse/FLINK-18646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17161786#comment-17161786
]
Caizhi Weng commented on FLINK-18646:
-------------------------------------
I discover this problem when running TPCDS 10T benchmark. The {{System.gc()}}
in the infinite loop of {{UnsafeMemoryBudget#reserveMemory}} is called about
400 times before the task slot is released.
I think this issue is caused by the usage of
{{JavaGcCleanerWrapper.tryRunPendingCleaners()}}. In Java8,
{{JavaGcCleanerWrapper.tryRunPendingCleaners()}} actually calls
{{java.lang.ref.Reference.tryHandlePending(false)}} and it only runs 1 cleaner
each call. If there are lots of pending cleaners, the infinite loop containing
{{System.gc()}} might run multiple times before exiting (the cleaner of other
slots might run before the cleaner of the current slot). So it seems that we
need to change
{code:java}
if (!JavaGcCleanerWrapper.tryRunPendingCleaners()) {
Thread.sleep(sleepTime);
sleepTime <<= 1;
sleeps++;
}
{code}
to
{code:java}
while (JavaGcCleanerWrapper.tryRunPendingCleaners()) {}
Thread.sleep(sleepTime);
sleepTime <<= 1;
sleeps++;
{code}
Also, it will be better if the cleaner only cleans the segments in its own
thread.
> 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
>
> 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)