You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2022/06/21 03:58:49 UTC

[GitHub] [flink] masteryhx commented on a diff in pull request #19679: [FLINK-23143][state/changelog] Support state migration for ChangelogSā€¦

masteryhx commented on code in PR #19679:
URL: https://github.com/apache/flink/pull/19679#discussion_r902140260


##########
flink-state-backends/flink-statebackend-changelog/src/main/java/org/apache/flink/state/changelog/restore/ChangelogRestoreTarget.java:
##########
@@ -21,52 +21,143 @@
 import org.apache.flink.api.common.state.State;
 import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
 import org.apache.flink.runtime.state.CheckpointableKeyedStateBackend;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyGroupedInternalPriorityQueue;
 import org.apache.flink.runtime.state.Keyed;
-import org.apache.flink.runtime.state.KeyedStateBackend;
 import org.apache.flink.runtime.state.PriorityComparable;
-import org.apache.flink.runtime.state.PriorityQueueSetFactory;
 import org.apache.flink.runtime.state.heap.HeapPriorityQueueElement;
+import org.apache.flink.runtime.state.internal.InternalKvState;
+import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot;
 import org.apache.flink.runtime.state.metainfo.StateMetaInfoSnapshot.BackendStateType;
+import org.apache.flink.state.changelog.ChangelogKeyGroupedPriorityQueue;
 import org.apache.flink.state.changelog.ChangelogState;
+import org.apache.flink.state.changelog.ChangelogStateFactory;
+import org.apache.flink.state.changelog.KvStateChangeLogger;
+import org.apache.flink.state.changelog.StateChangeLogger;
 
 import javax.annotation.Nonnull;
 
 /** Maintains metadata operation related to Changelog recovery. */
 @Internal
-public interface ChangelogRestoreTarget<K> {
+public abstract class ChangelogRestoreTarget<K> {
+
+    protected final AbstractKeyedStateBackend<K> keyedStateBackend;
+
+    protected final ChangelogStateFactory changelogStateFactory;
+
+    protected final FunctionDelegationHelper functionDelegationHelper;
+
+    public ChangelogRestoreTarget(
+            AbstractKeyedStateBackend<K> keyedStateBackend,
+            ChangelogStateFactory changelogStateFactory) {
+        this.keyedStateBackend = keyedStateBackend;
+        this.changelogStateFactory = changelogStateFactory;
+        this.functionDelegationHelper = new FunctionDelegationHelper();
+    }
 
     /** Returns the key groups which this restore procedure covers. */
-    KeyGroupRange getKeyGroupRange();
+    public KeyGroupRange getKeyGroupRange() {
+        return keyedStateBackend.getKeyGroupRange();
+    }
+
+    /**
+     * Returns the existing state created by {@link #createKeyedState(TypeSerializer,
+     * StateDescriptor)} or {@link #createPqState(String, TypeSerializer)} in the restore procedure.
+     */
+    public ChangelogState getExistingState(
+            String name, StateMetaInfoSnapshot.BackendStateType type) {
+        return changelogStateFactory.getExistingState(name, type);
+    }
 
     /**
      * Creates a keyed state which could be retrieved by {@link #getExistingState(String,
-     * BackendStateType)} in the restore procedure. The interface comes from {@link
-     * KeyedStateBackend#getOrCreateKeyedState(TypeSerializer, StateDescriptor)}.
+     * BackendStateType)} in the restore procedure.
      */
-    <N, S extends State, V> S createKeyedState(
+    @SuppressWarnings("unchecked")
+    public <N, S extends State, V> S createKeyedState(
             TypeSerializer<N> namespaceSerializer, StateDescriptor<S, V> stateDescriptor)
-            throws Exception;
+            throws Exception {
+        ChangelogState existingState =
+                changelogStateFactory.getExistingState(
+                        stateDescriptor.getName(),
+                        StateMetaInfoSnapshot.BackendStateType.KEY_VALUE);
+        if (existingState == null
+                || !isCompleteCompatible(
+                        (InternalKvState<K, N, V>) existingState,
+                        namespaceSerializer,
+                        stateDescriptor)) {
+            S keyedState =
+                    keyedStateBackend.upgradeKeyedState(namespaceSerializer, stateDescriptor);
+            functionDelegationHelper.addOrUpdate(stateDescriptor);
+            final InternalKvState<K, N, V> kvState = (InternalKvState<K, N, V>) keyedState;
+            return (S)
+                    changelogStateFactory.create(
+                            stateDescriptor,
+                            kvState,
+                            getKvStateChangeLogger(kvState, stateDescriptor),
+                            keyedStateBackend);
+        }
+        return (S) existingState;
+    }
+
+    private <N, S extends State, V> boolean isCompleteCompatible(
+            InternalKvState<K, N, V> existingState,
+            TypeSerializer<N> namespaceSerializer,
+            StateDescriptor<S, V> stateDescriptor) {
+        return isCompleteCompatible(existingState.getNamespaceSerializer(), namespaceSerializer)
+                && isCompleteCompatible(
+                        existingState.getValueSerializer(), stateDescriptor.getSerializer());
+    }
+
+    private <T> boolean isCompleteCompatible(
+            TypeSerializer<T> originalSerializer, TypeSerializer<T> newSerializer) {
+        return originalSerializer.equals(newSerializer);

Review Comment:
   I think `TypeSerializerSchemaCompatibility.Type.COMPATIBLE_AS_IS` is not enough for the case.
   We may also need to update the serializer in RocksDB/HashMap even if the compatibility is `COMPATIBLE_AS_IS` as you could see in [https://github.com/apache/flink/blob/99c2a415e9eeefafacf70762b6f54070f7911ceb/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateSerializerProvider.java#L314](url).
   Using equals because IIUC:
   1. If one serializer is equals the other, they must be `COMPATIBLE_AS_IS`, and no need to update the serializer.
   2. If one serializer `COMPATIBLE_AS_IS` with the other, they are not necessarily equal, and we need to update the serializer.
   
   WDYT?



-- 
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-unsubscribe@flink.apache.org

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