rkhachatryan commented on a change in pull request #18391:
URL: https://github.com/apache/flink/pull/18391#discussion_r794374746



##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendHandle.java
##########
@@ -130,14 +155,51 @@ public String toString() {
                     keyGroupRange, materialized.size(), 
nonMaterialized.size());
         }
 
-        private static Closeable asCloseable(KeyedStateHandle h) {
-            return () -> {
-                try {
-                    h.discardState();
-                } catch (Exception e) {
-                    ExceptionUtils.rethrowIOException(e);
+        private static class StreamStateHandleWrapper implements 
StreamStateHandle {
+            private static final long serialVersionUID = 1L;
+
+            private final KeyedStateHandle keyedStateHandle;
+
+            StreamStateHandleWrapper(KeyedStateHandle keyedStateHandle) {
+                this.keyedStateHandle = keyedStateHandle;
+            }
+
+            @Override
+            public void discardState() throws Exception {
+                keyedStateHandle.discardState();
+            }
+
+            @Override
+            public long getStateSize() {
+                return keyedStateHandle.getStateSize();
+            }
+
+            @Override
+            public FSDataInputStream openInputStream() throws IOException {
+                throw new UnsupportedOperationException("Should not call 
here.");
+            }
+
+            @Override
+            public Optional<byte[]> asBytesIfInMemory() {
+                throw new UnsupportedOperationException("Should not call 
here.");
+            }

Review comment:
       Maybe add a comment here about FLINK-25862 which you've created?
   Or mention this code in the ticket itself, so we don't forget about this 
place.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendHandle.java
##########
@@ -154,14 +177,51 @@ public String toString() {
                     keyGroupRange, materialized.size(), 
nonMaterialized.size());
         }
 
-        private static Closeable asCloseable(KeyedStateHandle h) {
-            return () -> {
-                try {
-                    h.discardState();
-                } catch (Exception e) {
-                    ExceptionUtils.rethrowIOException(e);
+        private static class StreamStateHandleWrapper implements 
StreamStateHandle {
+            private static final long serialVersionUID = 1L;
+
+            private final KeyedStateHandle keyedStateHandle;
+
+            StreamStateHandleWrapper(KeyedStateHandle keyedStateHandle) {
+                this.keyedStateHandle = keyedStateHandle;
+            }
+
+            @Override
+            public void discardState() throws Exception {
+                keyedStateHandle.discardState();
+            }
+
+            @Override
+            public long getStateSize() {
+                return keyedStateHandle.getStateSize();
+            }
+
+            @Override
+            public FSDataInputStream openInputStream() throws IOException {
+                throw new UnsupportedOperationException("Should not call 
here.");
+            }
+
+            @Override
+            public Optional<byte[]> asBytesIfInMemory() {
+                throw new UnsupportedOperationException("Should not call 
here.");
+            }
+
+            @Override
+            public boolean equals(Object o) {

Review comment:
       I guess we don't need `equals`/`hashCode` anymore?
   I usually assume the class is used as a key or is compared for equality when 
I encounter these methods.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendHandle.java
##########
@@ -44,28 +49,29 @@
  */
 @Internal
 public interface ChangelogStateBackendHandle extends KeyedStateHandle {
-    List<KeyedStateHandle> getMaterializedStateHandles();
+    Map<UUID, KeyedStateHandle> getMaterializedStateHandles();

Review comment:
       WDYT about using `StateHandleID` or `SharedStateRegistryKey` instead of 
`UUID` here and in other places?
   That would be more flexible I think (allowing to change ID generation); and 
also more consistent with the current code - 
`IncrementalRemoteKeyedStateHandle` uses `StateHandleID` as keys in its map.

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/state/changelog/ChangelogStateBackendHandle.java
##########
@@ -100,11 +115,19 @@ public KeyedStateHandle getIntersection(KeyGroupRange 
keyGroupRange) {
             if (intersection.getNumberOfKeyGroups() == 0) {
                 return null;
             }
-            List<KeyedStateHandle> basePart =
-                    this.materialized.stream()
-                            .map(handle -> 
handle.getIntersection(keyGroupRange))
-                            .filter(Objects::nonNull)
-                            .collect(Collectors.toList());
+            Map<UUID, KeyedStateHandle> basePart =
+                    this.materialized.entrySet().stream()
+                            .map(
+                                    entry ->
+                                            new AbstractMap.SimpleEntry<>(
+                                                    entry.getKey(),
+                                                    entry.getValue()
+                                                            
.getIntersection(keyGroupRange)))
+                            .filter(e -> e.getValue() != null)
+                            .collect(
+                                    Collectors.toMap(
+                                            AbstractMap.SimpleEntry::getKey,
+                                            
AbstractMap.SimpleEntry::getValue));

Review comment:
       nit: How about using `Tuple2` instead of `AbstractMap.SimpleEntry`?

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogStateBackend.java
##########
@@ -271,7 +273,7 @@ public StateBackend configure(ReadableConfig config, 
ClassLoader classLoader)
                                 keyedStateHandle instanceof 
ChangelogStateBackendHandle
                                         ? (ChangelogStateBackendHandle) 
keyedStateHandle
                                         : new ChangelogStateBackendHandleImpl(
-                                                
singletonList(keyedStateHandle),
+                                                
singletonMap(UUID.randomUUID(), keyedStateHandle),

Review comment:
       I see two potential problems here:
   1. When up-scaling, the same handle (file) may get multipiple keys; and 
therefore can be discarded at some point while still in use
   1. The original checkpoint does not register it's private state at all 
(`CompletedCheckpoint.registerSharedStatesAfterRestored`); so the original 
problem remains for the migration case
   
   Am I missing something?
   
   The only solution I see is to move this logic (`new 
ChangelogStateBackendHandleImpl`) to the JM.
   However, this doesn't sound right because the logic is backend-specific.




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