lindong28 commented on code in PR #20454: URL: https://github.com/apache/flink/pull/20454#discussion_r945555152
########## flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java: ########## @@ -312,26 +449,80 @@ 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()) Review Comment: Should we use `whenCompleteAsync()` here? ########## flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorHolder.java: ########## @@ -302,6 +377,68 @@ private void checkpointCoordinatorInternal( final long checkpointId, final CompletableFuture<byte[]> result) { mainThreadExecutor.assertRunningInMainThread(); + try { + if (currentPendingCheckpointId != OperatorCoordinator.NO_CHECKPOINT) { + 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)); + } + + subtaskGatewayMap.forEach( + (subtask, gateway) -> gateway.markForCheckpoint(checkpointId)); + + Preconditions.checkState(acknowledgeCloseGatewayFutureMap.isEmpty()); + } catch (Throwable 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); Review Comment: Should we use `whenCompleteAsync()` with `mainThreadExecutor`? -- 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