zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] implement channel state persistence for unaligned checkpoints URL: https://github.com/apache/flink/pull/11515#discussion_r401368470
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/SubtaskCheckpointCoordinatorImpl.java ########## @@ -89,17 +101,98 @@ public void checkpointState( checkpointMetaData.getCheckpointId(), checkpointOptions.getTargetLocation()); - CheckpointingOperation.execute( - checkpointMetaData, - checkpointOptions, - checkpointMetrics, - storage, - operatorChain, - taskName, - closeableRegistry, - executorService, - env, - asyncExceptionHandler, - isCanceled); + long startSyncPartNano = System.nanoTime(); + + HashMap<OperatorID, OperatorSnapshotFutures> operatorSnapshotsInProgress = new HashMap<>(operatorChain.getNumberOfOperators()); + try { + for (StreamOperatorWrapper<?, ?> operatorWrapper : operatorChain.getAllOperators(true)) { + StreamOperator<?> op = operatorWrapper.getStreamOperator(); + OperatorSnapshotFutures snapshotInProgress = checkpointStreamOperator( + op, + checkpointMetaData, + checkpointOptions, + storage, + isCanceled); + operatorSnapshotsInProgress.put(op.getOperatorID(), snapshotInProgress); + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Finished synchronous checkpoints for checkpoint {} on task {}", + checkpointMetaData.getCheckpointId(), taskName); + } + + long startAsyncPartNano = System.nanoTime(); + + checkpointMetrics.setSyncDurationMillis((startAsyncPartNano - startSyncPartNano) / 1_000_000); + + // we are transferring ownership over snapshotInProgressList for cleanup to the thread, active on submit + executorService.execute(new AsyncCheckpointRunnable( + operatorSnapshotsInProgress, + checkpointMetaData, + checkpointMetrics, + startAsyncPartNano, + taskName, + closeableRegistry, + env, + asyncExceptionHandler)); + + if (LOG.isDebugEnabled()) { + LOG.debug( + "{} - finished synchronous part of checkpoint {}. Alignment duration: {} ms, snapshot duration {} ms", + taskName, checkpointMetaData.getCheckpointId(), + checkpointMetrics.getAlignmentDurationNanos() / 1_000_000, + checkpointMetrics.getSyncDurationMillis()); + } + } catch (Exception ex) { + // Cleanup to release resources + for (OperatorSnapshotFutures operatorSnapshotResult : operatorSnapshotsInProgress.values()) { + if (null != operatorSnapshotResult) { + try { + operatorSnapshotResult.cancel(); + } catch (Exception e) { + LOG.warn("Could not properly cancel an operator snapshot result.", e); + } + } + } + + if (LOG.isDebugEnabled()) { + LOG.debug( + "{} - did NOT finish synchronous part of checkpoint {}. Alignment duration: {} ms, snapshot duration {} ms", + taskName, checkpointMetaData.getCheckpointId(), + checkpointMetrics.getAlignmentDurationNanos() / 1_000_000, + checkpointMetrics.getSyncDurationMillis()); + } + + if (checkpointOptions.getCheckpointType().isSynchronous()) { + // in the case of a synchronous checkpoint, we always rethrow the exception, + // so that the task fails. + // this is because the intention is always to stop the job after this checkpointing + // operation, and without the failure, the task would go back to normal execution. + throw ex; + } else { + env.declineCheckpoint(checkpointMetaData.getCheckpointId(), ex); + } + } + } + + private static OperatorSnapshotFutures checkpointStreamOperator( + StreamOperator<?> op, + CheckpointMetaData checkpointMetaData, + CheckpointOptions checkpointOptions, + CheckpointStreamFactory storageLocation, + Supplier<Boolean> isCanceled) throws Exception { + try { + return op.snapshotState( + checkpointMetaData.getCheckpointId(), + checkpointMetaData.getTimestamp(), + checkpointOptions, + storageLocation); + } + catch (Exception ex) { + if (!isCanceled.get()) { + LOG.info(ex.getMessage(), ex); Review comment: nit: maybe `warn` instead of `info` and give some custom message to indicate which process causes the exception. ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services