You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by "Yun Tang (Jira)" <ji...@apache.org> on 2021/10/27 09:26:00 UTC

[jira] [Created] (FLINK-24667) Channel state writer would fail the task directly if meeting exception previously

Yun Tang created FLINK-24667:
--------------------------------

             Summary: Channel state writer would fail the task directly if meeting exception previously
                 Key: FLINK-24667
                 URL: https://issues.apache.org/jira/browse/FLINK-24667
             Project: Flink
          Issue Type: Bug
          Components: Runtime / Checkpointing, Runtime / Task
    Affects Versions: 1.13.3, 1.14.0
            Reporter: Yun Tang


Currently, if channel state writer come across exception when closing a file, such as meet exception during {{SubtaskCheckpointCoordinatorImpl#cancelAsyncCheckpointRunnable}}, it will exit the loop. However, in the following {{channelStateWriter#abort}} it will throw exception directly:


{code:java}
switched from RUNNING to FAILED with failure cause: java.io.IOException: java.lang.RuntimeException: unable to send request to worker
	at org.apache.flink.runtime.io.network.partition.consumer.InputChannel.checkError(InputChannel.java:228)
	at org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel.checkPartitionRequestQueueInitialized(RemoteInputChannel.java:735)
	at org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel.getNextBuffer(RemoteInputChannel.java:204)
	at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.waitAndGetNextData(SingleInputGate.java:651)
	at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.getNextBufferOrEvent(SingleInputGate.java:626)
	at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.pollNext(SingleInputGate.java:612)
	at org.apache.flink.runtime.taskmanager.InputGateWithMetrics.pollNext(InputGateWithMetrics.java:109)
	at org.apache.flink.streaming.runtime.io.checkpointing.CheckpointedInputGate.pollNext(CheckpointedInputGate.java:149)
	at org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.emitNext(AbstractStreamTaskNetworkInput.java:110)
	at org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:66)
	at org.apache.flink.streaming.runtime.io.StreamTwoInputProcessor.processInput(StreamTwoInputProcessor.java:98)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:424)
	at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:204)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:685)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.executeInvoke(StreamTask.java:640)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:651)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:624)
	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:798)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:585)
{code}

This is not expected as checkpoint failure should not lead to task failover each time.



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