[ 
https://issues.apache.org/jira/browse/FLINK-18601?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Echo Lee updated FLINK-18601:
-----------------------------
    Summary: TaskManager shut down often in standalone cluster  (was: 
Taskmanager shut down often in standalone cluster)

> TaskManager shut down often in standalone cluster
> -------------------------------------------------
>
>                 Key: FLINK-18601
>                 URL: https://issues.apache.org/jira/browse/FLINK-18601
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Task
>    Affects Versions: 1.10.0, 1.11.0
>            Reporter: Echo Lee
>            Priority: Major
>
> The situation we encountered is that the job is running on the standalone 
> cluster. Cancelling the job occasionally causes the taskmanager to shut down. 
> I am not sure whether this is a problem. and Some of the logs are as follows:
> {code:java}
> 2020-07-14 20:16:00.169 [Cancellation Watchdog for KeyedProcess -> (Filter -> 
> Map, Sink: FeatureSinkToES) (1/1) (8c10a884bf480714460860f9d0e6131e).] ERROR 
> org.apache.flink.runtime.taskexecutor.TaskExecutor  - Task did not exit 
> gracefully within 180 + seconds.
> org.apache.flink.util.FlinkRuntimeException: Task did not exit gracefully 
> within 180 + seconds.
>         at 
> org.apache.flink.runtime.taskmanager.Task$TaskCancelerWatchDog.run(Task.java:1572)
>         at java.base/java.lang.Thread.run(Thread.java:834)
> 2020-07-14 20:16:00.169 [Cancellation Watchdog for KeyedProcess -> (Filter -> 
> Map, Sink: FeatureSinkToES) (1/1) (8c10a884bf480714460860f9d0e6131e).] ERROR 
> org.apache.flink.runtime.taskexecutor.TaskManagerRunner  - Fatal error 
> occurred while executing the TaskManager. Shutting it down...
> org.apache.flink.util.FlinkRuntimeException: Task did not exit gracefully 
> within 180 + seconds.
>         at 
> org.apache.flink.runtime.taskmanager.Task$TaskCancelerWatchDog.run(Task.java:1572)
>         at java.base/java.lang.Thread.run(Thread.java:834)
> 2020-07-14 20:16:00.170 [flink-akka.actor.default-dispatcher-27] INFO  
> org.apache.flink.runtime.taskexecutor.TaskExecutor  - Stopping TaskExecutor 
> akka.tcp://flink@10.3.67.116:35867/user/rpc/taskmanager_0.
> 2020-07-14 20:16:00.170 [flink-akka.actor.default-dispatcher-27] INFO  
> org.apache.flink.runtime.taskexecutor.TaskExecutor  - Close ResourceManager 
> connection fb5cf1580e59ff78505facc7010b099c.
> 2020-07-14 20:16:00.195 [Canceler/Interrupts for KeyedProcess -> (Filter -> 
> Map, Sink: FeatureSinkToES) (1/1) (8c10a884bf480714460860f9d0e6131e).] WARN  
> org.apache.flink.runtime.taskmanager.Task  - Task 'KeyedProcess -> (Filter -> 
> Map, Sink: FeatureSinkToES) (1/1)' did not react to cancelling signal for 30 
> seconds, but is stuck in method:
>  
> org.apache.flink.streaming.connectors.elasticsearch.ElasticsearchSinkBase.snapshotState(ElasticsearchSinkBase.java:325)
> app//org.apache.flink.streaming.util.functions.StreamingFunctionUtils.trySnapshotFunctionState(StreamingFunctionUtils.java:120)
> app//org.apache.flink.streaming.util.functions.StreamingFunctionUtils.snapshotFunctionState(StreamingFunctionUtils.java:101)
> app//org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.snapshotState(AbstractUdfStreamOperator.java:90)
> app//org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.snapshotState(StreamOperatorStateHandler.java:186)
> app//org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.snapshotState(StreamOperatorStateHandler.java:156)
> app//org.apache.flink.streaming.api.operators.AbstractStreamOperator.snapshotState(AbstractStreamOperator.java:314)
> app//org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.checkpointStreamOperator(SubtaskCheckpointCoordinatorImpl.java:614)
> app//org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.buildOperatorSnapshotFutures(SubtaskCheckpointCoordinatorImpl.java:540)
> app//org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.takeSnapshotSync(SubtaskCheckpointCoordinatorImpl.java:507)
> app//org.apache.flink.streaming.runtime.tasks.SubtaskCheckpointCoordinatorImpl.checkpointState(SubtaskCheckpointCoordinatorImpl.java:266)
> app//org.apache.flink.streaming.runtime.tasks.StreamTask.lambda$performCheckpoint$5(StreamTask.java:892)
> app//org.apache.flink.streaming.runtime.tasks.StreamTask$$Lambda$3269/0x00007f3f8a9ca560.run(Unknown
>  Source)
> app//org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.runThrowing(StreamTaskActionExecutor.java:47)
> app//org.apache.flink.streaming.runtime.tasks.StreamTask.performCheckpoint(StreamTask.java:882)
> app//org.apache.flink.streaming.runtime.tasks.StreamTask.triggerCheckpointOnBarrier(StreamTask.java:850)
> app//org.apache.flink.streaming.runtime.io.CheckpointBarrierHandler.notifyCheckpoint(CheckpointBarrierHandler.java:113)
> app//org.apache.flink.streaming.runtime.io.CheckpointBarrierAligner.processBarrier(CheckpointBarrierAligner.java:137)
> app//org.apache.flink.streaming.runtime.io.CheckpointedInputGate.pollNext(CheckpointedInputGate.java:93)
> app//org.apache.flink.streaming.runtime.io.StreamTaskNetworkInput.emitNext(StreamTaskNetworkInput.java:158)
> app//org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:67)
> app//org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:345)
> app//org.apache.flink.streaming.runtime.tasks.StreamTask$$Lambda$2662/0x00007f3f8a57a458.runDefaultAction(Unknown
>  Source)
> app//org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxStep(MailboxProcessor.java:191)
> app//org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:181)
> app//org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:558)
> app//org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:530)
> app//org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:721)
> app//org.apache.flink.runtime.taskmanager.Task.run(Task.java:546)
> java.base@11.0.2/java.lang.Thread.run(Thread.java:834)
> {code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to