lindong28 commented on code in PR #20454: URL: https://github.com/apache/flink/pull/20454#discussion_r941377666
########## flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java: ########## @@ -312,25 +423,97 @@ private void checkpointCoordinatorInternal( } else if (closeGateways(checkpointId)) { completeCheckpointOnceEventsAreDone(checkpointId, result, success); } else { - // if we cannot close the gateway, this means the checkpoint - // has been aborted before, so the future is already - // completed exceptionally. but we try to complete it here - // again, just in case, as a safety net. + // if we cannot close the gateway, this means the checkpoint has + // been aborted before, so the future is already completed + // exceptionally. but we try to complete it here again, just in + // case, as a safety net. result.completeExceptionally( new FlinkException("Cannot close gateway")); } return null; }, mainThreadExecutor)); - try { - subtaskGatewayMap.forEach( - (subtask, gateway) -> gateway.markForCheckpoint(checkpointId)); - coordinator.checkpointCoordinator(checkpointId, coordinatorCheckpoint); - } catch (Throwable t) { - ExceptionUtils.rethrowIfFatalErrorOrOOM(t); - result.completeExceptionally(t); - globalFailureHandler.handleGlobalFailure(t); + FutureUtils.combineAll(acknowledgeCloseGatewayFutureMap.values()) + .handleAsync( + (success, failure) -> { + if (failure != null) { + result.completeExceptionally(failure); + } else { + try { + coordinator.checkpointCoordinator( + checkpointId, coordinatorCheckpoint); + } catch (Throwable t) { + ExceptionUtils.rethrowIfFatalErrorOrOOM(t); + result.completeExceptionally(t); + globalFailureHandler.handleGlobalFailure(t); + } + } + return null; + }, + mainThreadExecutor); + } + + private void abortAllPendingAcknowledgeCloseGatewayFutures(String message) { + if (acknowledgeCloseGatewayFutureMap.isEmpty()) { + return; Review Comment: Is this line needed? ########## flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java: ########## @@ -299,9 +352,67 @@ public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData } private void checkpointCoordinatorInternal( - final long checkpointId, final CompletableFuture<byte[]> result) { + long checkpointId, CompletableFuture<byte[]> result) { mainThreadExecutor.assertRunningInMainThread(); + try { + subtaskGatewayMap.forEach( + (subtask, gateway) -> gateway.markForCheckpoint(checkpointId)); + + if (currentPendingCheckpointId != NO_CHECKPOINT + && currentPendingCheckpointId != checkpointId) { + throw new IllegalStateException( + String.format( + "Cannot checkpoint coordinator for checkpoint %d, " + + "since checkpoint %d has already started.", + checkpointId, currentPendingCheckpointId)); + } + + if (latestAttemptedCheckpointId >= checkpointId) { + throw new IllegalStateException( + String.format( + "Regressing checkpoint IDs. Previous checkpointId = %d, new checkpointId = %d", + latestAttemptedCheckpointId, checkpointId)); + } + + Preconditions.checkState(acknowledgeCloseGatewayFutureMap.isEmpty()); + } catch (Throwable t) { + ExceptionUtils.rethrowIfFatalErrorOrOOM(t); + result.completeExceptionally(t); + globalFailureHandler.handleGlobalFailure(t); + return; + } + + currentPendingCheckpointId = checkpointId; + latestAttemptedCheckpointId = checkpointId; + + for (int subtask : subtaskGatewayMap.keySet()) { + acknowledgeCloseGatewayFutureMap.put(subtask, new CompletableFuture<>()); + final OperatorEvent closeGatewayEvent = new CloseGatewayEvent(checkpointId, subtask); + subtaskGatewayMap + .get(subtask) + .sendEventWithCallBackOnCompletion( + closeGatewayEvent, + (success, failure) -> { + if (failure != null) { + // If the close gateway event failed to reach the subtask for + // some reason, the coordinator would trigger a fail-over on + // the subtask if the subtask is still running. This behavior + // also guarantees 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. + if (!(failure instanceof TaskNotRunningException)) { + subtaskGatewayMap + .get(subtask) + .tryTriggerTaskFailover(closeGatewayEvent, failure); + } + + completeAcknowledgeCloseGatewayFuture(subtask, checkpointId); Review Comment: Should we use `completeAcknowledgeCloseGatewayFutureExceptionally()` here for consistency with e.g. `executionAttemptFailed()`? -- 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: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org