[
https://issues.apache.org/jira/browse/FLINK-16510?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17154445#comment-17154445
]
Maximilian Michels commented on FLINK-16510:
--------------------------------------------
If you look at the stacktrace in the description, you can see that the presence
of a graceful shutdown including shutdown hooks can block a forceful shutdown
via {{System.getRuntime().halt()}}. We had applied the fix that Thomas linked
to our Flink 1.8 version, once we moved to Flink 1.10, we hadn't reapplied the
fix and were running into this issue again. Hence, we reverted back to always
using Runtime#halt(). It is crucial for us that the shutdown always succeeds,
no matter what, otherwise the hosting K8s pod won't terminate and we are in an
idle state.
FLINK-17470 is unrelated because we do not use the bash scripts to stop the
node. Rather, the problem is that the TaskManager does not terminate itself
during fatal errors, OOM in our case. I noticed that there is a configuration
value
{{https://ci.apache.org/projects/flink/flink-docs-stable/ops/config.html#taskmanager-jvm-exit-on-oom}}
but I don't think it is effective because an OOM can potentially be thrown at
non Task-related places where it won't lead to a forceful shutdown.
Would it make sense to have something like {{taskmanager.jvm-exit-on-error}}
which always forcefully exit on fatal errors if set to {{true}}?
> Task manager safeguard shutdown may not be reliable
> ---------------------------------------------------
>
> Key: FLINK-16510
> URL: https://issues.apache.org/jira/browse/FLINK-16510
> Project: Flink
> Issue Type: Bug
> Components: Runtime / Task
> Reporter: Maximilian Michels
> Assignee: Maximilian Michels
> Priority: Major
>
> The {{JvmShutdownSafeguard}} does not always succeed but can hang when
> multiple threads attempt to shutdown the JVM. Apparently mixing
> {{System.exit()}} with ShutdownHooks and forcefully terminating the JVM via
> {{Runtime.halt()}} does not play together well:
> {noformat}
> "Jvm Terminator" #22 daemon prio=5 os_prio=0 tid=0x00007fb8e82f2800
> nid=0x5a96 runnable [0x00007fb35cffb000]
> java.lang.Thread.State: RUNNABLE
> at java.lang.Shutdown.$$YJP$$halt0(Native Method)
> at java.lang.Shutdown.halt0(Shutdown.java)
> at java.lang.Shutdown.halt(Shutdown.java:139)
> - locked <0x000000047ed67638> (a java.lang.Shutdown$Lock)
> at java.lang.Runtime.halt(Runtime.java:276)
> at
> org.apache.flink.runtime.util.JvmShutdownSafeguard$DelayedTerminator.run(JvmShutdownSafeguard.java:86)
> at java.lang.Thread.run(Thread.java:748)
> Locked ownable synchronizers:
> - None
> "FlinkCompletableFutureDelayScheduler-thread-1" #18154 daemon prio=5
> os_prio=0 tid=0x00007fb708a7d000 nid=0x5a8a waiting for monitor entry
> [0x00007fb289d49000]
> java.lang.Thread.State: BLOCKED (on object monitor)
> at java.lang.Shutdown.halt(Shutdown.java:139)
> - waiting to lock <0x000000047ed67638> (a java.lang.Shutdown$Lock)
> at java.lang.Shutdown.exit(Shutdown.java:213)
> - locked <0x000000047edb7348> (a java.lang.Class for java.lang.Shutdown)
> at java.lang.Runtime.exit(Runtime.java:110)
> at java.lang.System.exit(System.java:973)
> at
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner.terminateJVM(TaskManagerRunner.java:266)
> at
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner.lambda$onFatalError$1(TaskManagerRunner.java:260)
> at
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner$$Lambda$27464/1464672548.accept(Unknown
> Source)
> at
> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:774)
> at
> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:750)
> at
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:488)
> at
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1990)
> at
> org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:943)
> at
> org.apache.flink.runtime.concurrent.DirectExecutorService.execute(DirectExecutorService.java:211)
> at
> org.apache.flink.runtime.concurrent.FutureUtils.lambda$orTimeout$11(FutureUtils.java:361)
> at
> org.apache.flink.runtime.concurrent.FutureUtils$$Lambda$27435/159015392.run(Unknown
> Source)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$201(ScheduledThreadPoolExecutor.java:180)
> at
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:293)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:748)
> Locked ownable synchronizers:
> - <0x00000006d5e56bd0> (a
> java.util.concurrent.ThreadPoolExecutor$Worker)
> {noformat}
> Note that under this condition the JVM should terminate but it still hangs.
> Sometimes it quits after several minutes.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)