curcur commented on a change in pull request #15200:
URL: https://github.com/apache/flink/pull/15200#discussion_r646260894



##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogMapState.java
##########
@@ -71,22 +94,67 @@ public boolean contains(UK key) throws Exception {
 
     @Override
     public Iterable<Map.Entry<UK, UV>> entries() throws Exception {
-        return delegatedState.entries();
+        Iterator<Map.Entry<UK, UV>> iterator = 
delegatedState.entries().iterator();
+        final N currentNamespace = getCurrentNamespace();
+        return () ->
+                loggingIterator(
+                        new Iterator<Map.Entry<UK, UV>>() {
+                            @Override
+                            public Map.Entry<UK, UV> next() {
+                                return loggingMapEntry(
+                                        iterator.next(),
+                                        changeLogger,
+                                        changeWriter,
+                                        currentNamespace);
+                            }
+
+                            @Override
+                            public boolean hasNext() {
+                                return iterator.hasNext();
+                            }
+
+                            @Override
+                            public void remove() {
+                                iterator.remove();
+                            }
+                        },
+                        changeLogger,
+                        changeWriter,
+                        currentNamespace);
     }
 
     @Override
     public Iterable<UK> keys() throws Exception {
-        return delegatedState.keys();
+        return loggingIterable(
+                delegatedState.keys(), changeLogger, this::serializeKey, 
getCurrentNamespace());

Review comment:
       Sync up offline:
   
   Let me rephrase my question:
   
   1. Some backend supports iterator change/remove; but some does not, but this 
is not the focus, I am asking in the case where iterator change/remove are 
supported (rocksdb for example)
   
   2. In the case where iterator is supported
   Let's say I get an iterator list of keys, and I get an iterator list of 
values, I remove some keys in the key iterator, and some values in the value 
iterator
   
   Is it guaranteed that such remove is always paired? (Key, Value)




-- 
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:
[email protected]


Reply via email to