zoltar9264 commented on code in PR #22669:
URL: https://github.com/apache/flink/pull/22669#discussion_r1212507940


##########
flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/snapshot/RocksDBSnapshotStrategyBase.java:
##########
@@ -395,18 +394,16 @@ public void release() {
     /** Previous snapshot with uploaded sst files. */
     protected static class PreviousSnapshot {
 
-        @Nullable private final Map<StateHandleID, Long> confirmedSstFiles;
+        @Nullable private final Map<StateHandleID, StreamStateHandle> 
confirmedSstFiles;
 
-        protected PreviousSnapshot(@Nullable Map<StateHandleID, Long> 
confirmedSstFiles) {
+        protected PreviousSnapshot(
+                @Nullable Map<StateHandleID, StreamStateHandle> 
confirmedSstFiles) {
             this.confirmedSstFiles = confirmedSstFiles;
         }
 
         protected Optional<StreamStateHandle> getUploaded(StateHandleID 
stateHandleID) {
             if (confirmedSstFiles != null && 
confirmedSstFiles.containsKey(stateHandleID)) {
-                // we introduce a placeholder state handle, that is replaced 
with the
-                // original from the shared state registry (created from a 
previous checkpoint)
-                return Optional.of(
-                        new 
PlaceholderStreamStateHandle(confirmedSstFiles.get(stateHandleID)));

Review Comment:
   I really don't have a full test to prove that removing 
PlaceholderStreamStateHandle has no regression, I therefore agree to keep it. 
But I still recommend limiting its usage to ByteStreamStateHandle, and don't 
use it to calculate checkpointed size.
   And I suggest saving the PhysicalStateHandleID of the original StateHandle 
into the PlaceholderStreamStateHandle to simplify the registration. And we can 
check it before replacing.
   WDYT @rkhachatryan ?



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