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



##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackend.java
##########
@@ -328,14 +340,23 @@ public void notifyCheckpointAborted(long checkpointId) 
throws Exception {
             throw new FlinkRuntimeException(message);
         }
 
-        return stateFactory.create(
+        InternalKvState<K, N, SV> state =
                 keyedStateBackend.createInternalState(
-                        namespaceSerializer, stateDesc, 
snapshotTransformFactory));
+                        namespaceSerializer, stateDesc, 
snapshotTransformFactory);
+        KvStateChangeLoggerImpl<K, SV, N> logger =

Review comment:
       -> `KvStateChangeLogger`

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/AbstractStateChangeLogger.java
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.runtime.state.changelog.StateChangelogWriter;
+import org.apache.flink.runtime.state.heap.InternalKeyContext;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.ADD;
+import static 
org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.CHANGE_ELEMENT;
+import static 
org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.CLEAR;
+import static 
org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.REMOVE_ELEMENT;
+import static 
org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.SET;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+abstract class AbstractStateChangeLogger<Key, State, Ns> implements 
StateChangeLogger<State, Ns> {
+    protected final StateChangelogWriter<?> stateChangelogWriter;
+    protected final InternalKeyContext<Key> keyContext;
+
+    public AbstractStateChangeLogger(
+            StateChangelogWriter<?> stateChangelogWriter, 
InternalKeyContext<Key> keyContext) {
+        this.stateChangelogWriter = checkNotNull(stateChangelogWriter);
+        this.keyContext = checkNotNull(keyContext);
+    }
+
+    @Override
+    public void stateUpdated(State newState, Ns ns) throws IOException {
+        if (newState == null) {
+            stateCleared(ns);
+        } else {
+            log(SET, out -> serializeState(newState, out), ns);
+        }
+    }
+
+    protected abstract void serializeState(State state, 
DataOutputViewStreamWrapper out)
+            throws IOException;
+
+    @Override
+    public void stateAdded(State addedState, Ns ns) throws IOException {
+        log(ADD, out -> serializeState(addedState, out), ns);
+    }
+
+    @Override
+    public void stateCleared(Ns ns) throws IOException {
+        log(CLEAR, out -> {}, ns);
+    }
+
+    @Override
+    public void stateElementChanged(
+            ThrowingConsumer<DataOutputViewStreamWrapper, IOException> 
dataSerializer, Ns ns)
+            throws IOException {
+        log(CHANGE_ELEMENT, dataSerializer, ns);
+    }
+
+    @Override
+    public void stateElementRemoved(
+            ThrowingConsumer<DataOutputViewStreamWrapper, IOException> 
dataSerializer, Ns ns)
+            throws IOException {
+        log(REMOVE_ELEMENT, dataSerializer, ns);
+    }
+
+    protected void log(
+            StateChangeOperation op,
+            ThrowingConsumer<DataOutputViewStreamWrapper, IOException> 
dataWriter,
+            Ns ns)
+            throws IOException {
+        // todo: log metadata (FLINK-22808)
+        stateChangelogWriter.append(
+                keyContext.getCurrentKeyGroupIndex(), serialize(op, ns, 
dataWriter));
+    }
+
+    private byte[] serialize(
+            StateChangeOperation op,
+            Ns ns,
+            ThrowingConsumer<DataOutputViewStreamWrapper, IOException> 
dataWriter)
+            throws IOException {
+        return serializeRaw(
+                wrapper -> {
+                    wrapper.writeByte(op.code);
+                    serializeScope(ns, wrapper);
+                    dataWriter.accept(wrapper);
+                });
+    }
+
+    protected abstract void serializeScope(Ns ns, DataOutputViewStreamWrapper 
out)
+            throws IOException;
+
+    private byte[] serializeRaw(
+            ThrowingConsumer<DataOutputViewStreamWrapper, IOException> 
dataWriter)
+            throws IOException {
+        try (ByteArrayOutputStream out = new ByteArrayOutputStream();
+                DataOutputViewStreamWrapper wrapper = new 
DataOutputViewStreamWrapper(out)) {
+            dataWriter.accept(wrapper);
+            return out.toByteArray();
+        }
+    }
+
+    enum StateChangeOperation {
+        /** Scope: key + namespace. */
+        CLEAR((byte) 0),
+        /** Scope: key + namespace. */
+        SET((byte) 1),
+        /** Scope: key + namespace. */
+        ADD((byte) 2),
+        /** Scope: key + namespace, also affecting other (source) namespaces. 
*/
+        MERGE_NS((byte) 3),
+        /** Scope: key + namespace + element (e.g. user map key put or list 
append). */
+        CHANGE_ELEMENT((byte) 4),
+        /** Scope: key + namespace + element (e.g. user map remove or iterator 
remove). */
+        REMOVE_ELEMENT((byte) 5),
+        /** Scope: key + namespace, last element (e.g. priority queue poll). */
+        POLL_ELEMENT((byte) 6);
+        private final byte code;
+
+        StateChangeOperation(byte code) {
+            this.code = code;
+        }
+
+        static final Map<Byte, KvStateChangeLoggerImpl.StateChangeOperation> 
BY_CODES =

Review comment:
       Sorry to be picky, but make it `private`?

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogKeyedStateBackend.java
##########
@@ -248,8 +253,15 @@ public boolean 
deregisterKeySelectionListener(KeySelectionListener<K> listener)
             KeyGroupedInternalPriorityQueue<T> create(
                     @Nonnull String stateName,
                     @Nonnull TypeSerializer<T> byteOrderedElementSerializer) {
-        return new ChangelogKeyGroupedPriorityQueue<T>(
-                keyedStateBackend.create(stateName, 
byteOrderedElementSerializer));
+        PriorityQueueStateChangeLoggerImpl<K, T> logger =

Review comment:
       -> `PriorityQueueStateChangeLogger`

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogListState.java
##########
@@ -39,53 +44,83 @@
         extends AbstractChangelogState<K, N, List<V>, InternalListState<K, N, 
V>>
         implements InternalListState<K, N, V> {
 
-    ChangelogListState(InternalListState<K, N, V> delegatedState) {
-        super(delegatedState);
+    ChangelogListState(
+            InternalListState<K, N, V> delegatedState,
+            KvStateChangeLogger<List<V>, N> changeLogger) {
+        super(delegatedState, changeLogger);
     }
 
     @Override
     public void update(List<V> values) throws Exception {
+        changeLogger.stateUpdated(values, getCurrentNamespace());
         delegatedState.update(values);
     }
 
     @Override
     public void addAll(List<V> values) throws Exception {
+        changeLogger.stateAdded(values, getCurrentNamespace());
         delegatedState.addAll(values);
     }
 
     @Override
     public void updateInternal(List<V> valueToStore) throws Exception {
+        changeLogger.stateUpdated(valueToStore, getCurrentNamespace());
         delegatedState.updateInternal(valueToStore);
     }

Review comment:
       How is `update` different from `updateInternal`
   
   From reading their comments, it seems they are different somehow, but the 
changelogger seems to be same.
   
   Similar to other states.
   
   

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogAggregatingState.java
##########
@@ -67,18 +71,20 @@ public OUT get() throws Exception {
     @Override
     public void add(IN value) throws Exception {
         delegatedState.add(value);
+        changeLogger.stateUpdated(delegatedState.getInternal(), 
getCurrentNamespace());

Review comment:
       Sync up offline:
   
   Yes, I got it when I see this . I do not have better ways to do this other 
than the current way, but you get why I mentioned this right? Conceptually, we 
are expected to log the incremental action log instead of the updated value.
   
   Let's sync up this offline

##########
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:
       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)

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/ChangelogListState.java
##########
@@ -39,53 +44,83 @@
         extends AbstractChangelogState<K, N, List<V>, InternalListState<K, N, 
V>>
         implements InternalListState<K, N, V> {
 
-    ChangelogListState(InternalListState<K, N, V> delegatedState) {
-        super(delegatedState);
+    ChangelogListState(
+            InternalListState<K, N, V> delegatedState,
+            KvStateChangeLogger<List<V>, N> changeLogger) {
+        super(delegatedState, changeLogger);
     }
 
     @Override
     public void update(List<V> values) throws Exception {
+        changeLogger.stateUpdated(values, getCurrentNamespace());
         delegatedState.update(values);
     }
 
     @Override
     public void addAll(List<V> values) throws Exception {
+        changeLogger.stateAdded(values, getCurrentNamespace());
         delegatedState.addAll(values);
     }
 
     @Override
     public void updateInternal(List<V> valueToStore) throws Exception {
+        changeLogger.stateUpdated(valueToStore, getCurrentNamespace());
         delegatedState.updateInternal(valueToStore);
     }
 
     @Override
     public void add(V value) throws Exception {
+        if (getValueSerializer() instanceof ListSerializer) {
+            changeLogger.stateElementChanged(
+                    w ->
+                            ((ListSerializer<V>) getValueSerializer())
+                                    .getElementSerializer()
+                                    .serialize(value, w),
+                    getCurrentNamespace());
+        } else {
+            changeLogger.stateAdded(singletonList(value), 
getCurrentNamespace());
+        }

Review comment:
       Sync up offline.
   
   I am confused with this part.
   
    if (getValueSerializer() instanceof ListSerializer    is not true,
   adding singletonList will also be a problem?
   
   That says when would be the case where getValueSerializer() is not ready
   but `changeLogger.stateAdded(singletonList(value), getCurrentNamespace());` 
can be successful.
   
   

##########
File path: 
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/AbstractStateChangeLogger.java
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.runtime.state.changelog.StateChangelogWriter;
+import org.apache.flink.runtime.state.heap.InternalKeyContext;
+import org.apache.flink.util.function.ThrowingConsumer;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import static 
org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.ADD;
+import static 
org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.CHANGE_ELEMENT;
+import static 
org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.CLEAR;
+import static 
org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.REMOVE_ELEMENT;
+import static 
org.apache.flink.state.changelog.AbstractStateChangeLogger.StateChangeOperation.SET;
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+abstract class AbstractStateChangeLogger<Key, State, Ns> implements 
StateChangeLogger<State, Ns> {
+    protected final StateChangelogWriter<?> stateChangelogWriter;
+    protected final InternalKeyContext<Key> keyContext;
+
+    public AbstractStateChangeLogger(
+            StateChangelogWriter<?> stateChangelogWriter, 
InternalKeyContext<Key> keyContext) {
+        this.stateChangelogWriter = checkNotNull(stateChangelogWriter);
+        this.keyContext = checkNotNull(keyContext);
+    }
+
+    @Override
+    public void stateUpdated(State newState, Ns ns) throws IOException {
+        if (newState == null) {
+            stateCleared(ns);
+        } else {
+            log(SET, out -> serializeState(newState, out), ns);
+        }
+    }
+
+    protected abstract void serializeState(State state, 
DataOutputViewStreamWrapper out)
+            throws IOException;
+
+    @Override
+    public void stateAdded(State addedState, Ns ns) throws IOException {
+        log(ADD, out -> serializeState(addedState, out), ns);
+    }
+
+    @Override
+    public void stateCleared(Ns ns) throws IOException {
+        log(CLEAR, out -> {}, ns);
+    }
+
+    @Override
+    public void stateElementChanged(
+            ThrowingConsumer<DataOutputViewStreamWrapper, IOException> 
dataSerializer, Ns ns)
+            throws IOException {
+        log(CHANGE_ELEMENT, dataSerializer, ns);
+    }
+
+    @Override
+    public void stateElementRemoved(
+            ThrowingConsumer<DataOutputViewStreamWrapper, IOException> 
dataSerializer, Ns ns)
+            throws IOException {
+        log(REMOVE_ELEMENT, dataSerializer, ns);
+    }
+
+    protected void log(
+            StateChangeOperation op,
+            ThrowingConsumer<DataOutputViewStreamWrapper, IOException> 
dataWriter,
+            Ns ns)
+            throws IOException {
+        // todo: log metadata (FLINK-22808)
+        stateChangelogWriter.append(
+                keyContext.getCurrentKeyGroupIndex(), serialize(op, ns, 
dataWriter));
+    }
+
+    private byte[] serialize(
+            StateChangeOperation op,
+            Ns ns,
+            ThrowingConsumer<DataOutputViewStreamWrapper, IOException> 
dataWriter)
+            throws IOException {
+        return serializeRaw(
+                wrapper -> {
+                    wrapper.writeByte(op.code);
+                    serializeScope(ns, wrapper);
+                    dataWriter.accept(wrapper);
+                });
+    }
+
+    protected abstract void serializeScope(Ns ns, DataOutputViewStreamWrapper 
out)
+            throws IOException;
+
+    private byte[] serializeRaw(
+            ThrowingConsumer<DataOutputViewStreamWrapper, IOException> 
dataWriter)
+            throws IOException {
+        try (ByteArrayOutputStream out = new ByteArrayOutputStream();
+                DataOutputViewStreamWrapper wrapper = new 
DataOutputViewStreamWrapper(out)) {
+            dataWriter.accept(wrapper);
+            return out.toByteArray();
+        }
+    }
+
+    enum StateChangeOperation {
+        /** Scope: key + namespace. */
+        CLEAR((byte) 0),
+        /** Scope: key + namespace. */
+        SET((byte) 1),
+        /** Scope: key + namespace. */
+        ADD((byte) 2),
+        /** Scope: key + namespace, also affecting other (source) namespaces. 
*/
+        MERGE_NS((byte) 3),
+        /** Scope: key + namespace + element (e.g. user map key put or list 
append). */
+        CHANGE_ELEMENT((byte) 4),
+        /** Scope: key + namespace + element (e.g. user map remove or iterator 
remove). */
+        REMOVE_ELEMENT((byte) 5),

Review comment:
       not strong opinion on this, I was thinking to make this more explicit 
for change, what do you think?
   
   UPDATE_OR_ADD
   APPEND
   REMOVE
   ....




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