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



##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogMapState.java
##########
@@ -96,12 +164,39 @@ public boolean isEmpty() throws Exception {
 
     @Override
     public void clear() {
+        try {
+            changeLogger.valueCleared(getCurrentNamespace());
+        } catch (IOException e) {
+            ExceptionUtils.rethrow(e);
+        }
         delegatedState.clear();
     }
 
+    private void serializeValue(
+            UV value, org.apache.flink.core.memory.DataOutputViewStreamWrapper 
out)
+            throws IOException {
+        getMapSerializer().getValueSerializer().serialize(value, out);
+    }
+
+    private void serializeKey(UK key, 
org.apache.flink.core.memory.DataOutputViewStreamWrapper out)
+            throws IOException {
+        getMapSerializer().getKeySerializer().serialize(key, out);
+    }
+
+    private MapSerializer<UK, UV> getMapSerializer() {
+        return (MapSerializer<UK, UV>) getValueSerializer();
+    }
+
+    private final BiConsumerWithException<
+                    Map.Entry<UK, UV>, DataOutputViewStreamWrapper, 
IOException>
+            changeWriter = (entry, out) -> serializeKey(entry.getKey(), out);

Review comment:
       Move this to the top of this class...
   
   Confusing to read.

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/StateChangeLogger.java
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.state.changelog;
+
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.function.BiConsumerWithException;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+
+interface StateChangeLogger<State, Namespace> {
+    static <Namespace, State, StateElement> Iterator<StateElement> 
loggingIterator(
+            @Nullable Iterator<StateElement> iterator,
+            StateChangeLogger<State, Namespace> changeLogger,
+            BiConsumerWithException<StateElement, DataOutputViewStreamWrapper, 
IOException>
+                    changeWriter,
+            Namespace ns) {
+        if (iterator == null) {
+            return null;
+        }
+        return new Iterator<StateElement>() {
+
+            @Nullable private StateElement lastReturned;
+
+            @Override
+            public boolean hasNext() {
+                return iterator.hasNext();
+            }
+
+            @Override
+            public StateElement next() {
+                return lastReturned = iterator.next();
+            }
+
+            @Override
+            public void remove() {
+                try {
+                    changeLogger.valueElementRemoved(
+                            out -> changeWriter.accept(lastReturned, out), ns);
+                } catch (IOException e) {
+                    ExceptionUtils.rethrow(e);
+                }
+                iterator.remove();
+            }
+        };
+    }
+
+    static <Namespace, State, StateElement> Iterable<StateElement> 
loggingIterable(
+            @Nullable Iterable<StateElement> iterable,
+            KvStateChangeLogger<State, Namespace> changeLogger,
+            BiConsumerWithException<StateElement, DataOutputViewStreamWrapper, 
IOException>
+                    changeWriter,
+            Namespace ns) {
+        if (iterable == null) {
+            return null;
+        }
+        return () -> loggingIterator(iterable.iterator(), changeLogger, 
changeWriter, ns);
+    }
+
+    static <UK, UV, State, Namespace> Map.Entry<UK, UV> loggingMapEntry(
+            Map.Entry<UK, UV> entry,
+            KvStateChangeLogger<State, Namespace> changeLogger,
+            BiConsumerWithException<Map.Entry<UK, UV>, 
DataOutputViewStreamWrapper, IOException>
+                    changeWriter,
+            Namespace ns) {
+        return new Map.Entry<UK, UV>() {
+            @Override
+            public UK getKey() {
+                return entry.getKey();
+            }
+
+            @Override
+            public UV getValue() {
+                return entry.getValue();
+            }
+
+            @Override
+            public UV setValue(UV value) {
+                try {
+                    changeLogger.valueElementChanged(out -> 
changeWriter.accept(entry, out), ns);

Review comment:
       Do you need to log the value as well?
   
   If you look at the how `changeWriter` is defined in ChangelogMapState, it 
only logs the key.
   
   ```
   private final BiConsumerWithException<
                       Map.Entry<UK, UV>, DataOutputViewStreamWrapper, 
IOException>
               changeWriter = (entry, out) -> serializeKey(entry.getKey(), out);
   ```

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/StateChangeLogger.java
##########
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.state.changelog;
+
+import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.function.BiConsumerWithException;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Map;
+
+interface StateChangeLogger<State, Namespace> {
+    static <Namespace, State, StateElement> Iterator<StateElement> 
loggingIterator(

Review comment:
       How about this:
   
   Keep `oggingIterator` and `loggingIterable` within `StateChangeLogger` 
interface. That makes sense for general iterator wrappers.
   
   Move `loggingMapEntry` to `ChangelogMapState`, because it is map 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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to