mayuehappy commented on code in PR #23765:
URL: https://github.com/apache/flink/pull/23765#discussion_r1404175347


##########
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksIncrementalSnapshotStrategy.java:
##########
@@ -350,6 +359,10 @@ private long uploadSnapshotFiles(
                                 ? CheckpointedStateScope.EXCLUSIVE
                                 : CheckpointedStateScope.SHARED;
 
+                if (stateFileVerifier != null) {
+                    stateFileVerifier.verifySstFilesChecksum(sstFilePaths);
+                }
+
                 List<HandleAndLocalPath> sstFilesUploadResult =
                         stateUploader.uploadFilesToCheckpointFs(

Review Comment:
   
   
   
   > From the code order, we verify the file first, and then 
`uploadFilesToCheckpointFs`.
   > 
   > If the file is corrupted before calling `hdfs.write()`, this check still 
cannot cover, right?
   
   
   Yes ,  If the file is corrupted before calling `hdfs.write()`, this check 
still cannot cover
   
   But as I understand,  After the file is hardlinked to snapshotDir, no other 
operations will change the file. So there should be no file corruption during 
this process.
   
   I think there are only two places where corruption may occur, one is when 
the file is created, and the other is when it is re-downloaded from Checkpoint 
to the local disk.
   



-- 
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

Reply via email to