[ https://issues.apache.org/jira/browse/FLINK-18646?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17161825#comment-17161825 ]
Andrey Zagrebin edited comment on FLINK-18646 at 7/21/20, 8:11 AM: ------------------------------------------------------------------- I do not think that {{System.gc()}} is actually doing something all the time, it is just a hint to JVM that it is allowed to spent more time for GC. Anyways we can indeed remove it to see whether it helps. If {{tryRunPendingCleaners == true}} there will be no {{sleeps}} and the outer loop is basically {{while (tryRunPendingCleaners) plus {{System.gc()}}, which is almost noop hint according to logs.}}. Also generally, it is not optimal for each reserve operation to wait for all cleaners by {{while (tryRunPendingCleaners)}} without trying to reserve because the reserve operation might want to reserve just a bit and needs to wait for a couple of cleaners. The cleaners are already supposed to clean in some GC background threads. By calling {{tryRunPendingCleaners}} , we just try to speed it up giving it another thread and more CPU if the available memory is exhausted. Therefore, it makes sense to allocate a thread for the check because otherwise it waits for the full GC of the slot managed memory in RPC thread. I would try this in {{TaskSlot}}: {code:java} CompletableFuture<Void> closeAsync(Throwable cause) { ..... ExecutorService executor = Executors.newSingleThreadExecutor(); // <------------ final CompletableFuture<Void> cleanupFuture = FutureUtils .waitForAll(tasks.values().stream().map(TaskSlotPayload::getTerminationFuture).collect(Collectors.toList())) .thenRunAsync(this::verifyMemoryFreed, executor) // <------------ .thenRunAsync(() -> { this.memoryManager.shutdown(); executor.shutdown(); // <------------ }); .... } {code} was (Author: azagrebin): I do not think that {{System.gc()}} is actually doing something all the time, it is just a hint to JVM that it is allowed to spent more time for GC. Anyways we can indeed remove it to see whether it helps. If {{tryRunPendingCleaners == true}} there will be no {{sleeps}} and the outer loop is basically {{while (tryRunPendingCleaners)}}. Also generally, it is not optimal for each reserve operation to wait for all cleaners by {{while (tryRunPendingCleaners)}} without trying to reserve because the reserve operation might want to reserve just a bit and needs to wait for a couple of cleaners. The cleaners are already supposed to clean in some GC background threads. By calling {{tryRunPendingCleaners}} , we just try to speed it up giving it another thread and more CPU if the available memory is exhausted. Therefore, it makes sense to allocate a thread for the check because otherwise it waits for the full GC of the slot managed memory in RPC thread. I would try this in {{TaskSlot}}: {code:java} CompletableFuture<Void> closeAsync(Throwable cause) { ..... ExecutorService executor = Executors.newSingleThreadExecutor(); // <------------ final CompletableFuture<Void> cleanupFuture = FutureUtils .waitForAll(tasks.values().stream().map(TaskSlotPayload::getTerminationFuture).collect(Collectors.toList())) .thenRunAsync(this::verifyMemoryFreed, executor) // <------------ .thenRunAsync(() -> { this.memoryManager.shutdown(); executor.shutdown(); // <------------ }); .... } {code} > 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)