rkhachatryan commented on code in PR #19679:
URL: https://github.com/apache/flink/pull/19679#discussion_r902024128
##########
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:
Is the contract of Serializers `equals` suitable for migration?
Shouldn't it be something like
`originalSerializer.snapshotConfiguration().resolveSchemaCompatibility(newSerializer).isIncompatible()`
?
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]