1996fanrui commented on code in PR #19993:
URL: https://github.com/apache/flink/pull/19993#discussion_r901261999
##########
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriteRequest.java:
##########
@@ -109,6 +112,9 @@ static ChannelStateWriteRequest buildFutureWriteRequest(
}
},
throwable -> {
+ if (!dataFuture.isDone()) {
+ return;
+ }
Review Comment:
Hi @zentol @pnowojski , as I understand, there are four places in
ChannelStateWriterThread that may be interrupted:
1. Write a file
2. deque.take() : pull request from deque
3. dataFuture.get() in action: Write the file normally.
4. dataFuture.get() in discardAction: Executed when canceling a request.
1,2 and 3 are all executed in ChannelStateWriteRequestExecutorImpl#loop, and
InterruptedException is caught in the loop, so 1, 2 and 3 will not cause job
InterruptedException. But case 4 may be called by
ChannelStateWriteRequestExecutorImpl#cleanupRequests without catching
InterruptedException, so case 4 may cause job InterruptedException(this is the
root cause of FLINK-27792).
## Why FLINK-28077? Why stuck in dataFuture.get() of the discardAction?
ChannelStateWriteRequestExecutorImpl#cleanupRequests will call the
discardAction of all rest requests. The thread.interrupt() takes effect only
once, if there are multiple requests that need to wait for dataFuture.get(),
only the first request will be interrupted, and the remaining requests will be
stuck(this is the root cause of FLINK-28077).
If it is changed to the proposal of @zentol , both FLINK-27792 and
FLINK-28077 will be resolved. I have updated the PR. And I improved the test:
`ChannelStateWriteRequestExecutorImplTest#testCanBeClosed`. For old code, the
unit-test gets stuck in the second `dataFuture.get()`. After the change, the
unit-test worked fine.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]