yunfengzhou-hub commented on code in PR #20454:
URL: https://github.com/apache/flink/pull/20454#discussion_r939861202
##########
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java:
##########
@@ -298,6 +322,86 @@ public void resetToCheckpoint(long checkpointId, @Nullable
byte[] checkpointData
coordinator.resetToCheckpoint(checkpointId, checkpointData);
}
+ private void closeGatewayAndCheckpointCoordinator(
+ long checkpointId, CompletableFuture<byte[]> result) {
+ mainThreadExecutor.assertRunningInMainThread();
+
+ latestAttemptedCheckpointId = checkpointId;
+
+ acknowledgeCloseGatewayFutureMap.clear();
+ for (int subtask : subtaskGatewayMap.keySet()) {
+ acknowledgeCloseGatewayFutureMap.put(subtask, new
CompletableFuture<>());
+ subtaskGatewayMap
+ .get(subtask)
+ .sendEventWithCallBackOnCompletion(
+ new CloseGatewayEvent(checkpointId),
+ (success, failure) -> {
+ if (failure != null) {
+ // The close gateway event failed to reach
the subtask for some
+ // reason. For example, the subtask has
finished. In this case
+ // it is guaranteed that the coordinator
won't receive more
+ // events from this subtask before the
current checkpoint
+ // finishes, which is equivalent to
receiving ACK from this
+ // subtask.
+
completeAcknowledgeCloseGatewayFuture(subtask, checkpointId);
+
+ if (!(failure instanceof
RejectedExecutionException
Review Comment:
As the infrastructure code introduced in this PR would send close gateway
events to all of a coordinator's subtasks for better simplicity on the API
changes, regardless of whether a subtask has finished or not, it would be
normal for the coordinator to receive exceptions related to tasks not in
running state. Thus we should explicitly catch this exception here. As the
subtasks are already not running when close gateway events are sent, there is
no need to trigger a failover on that subtask.
--
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]