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)