You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/10/31 15:40:06 UTC

[jira] [Commented] (FLINK-9376) Allow upgrading to incompatible state serializers (state schema evolution)

    [ https://issues.apache.org/jira/browse/FLINK-9376?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16670253#comment-16670253 ] 

ASF GitHub Bot commented on FLINK-9376:
---------------------------------------

aljoscha closed pull request #6325: [FLINK-9376] Allow upgrading to incompatible state serializers (state schema evolution)
URL: https://github.com/apache/flink/pull/6325
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java
index 161e65b3b06..bbf512eaf79 100644
--- a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java
+++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/api/java/typeutils/runtime/WritableSerializer.java
@@ -19,10 +19,10 @@
 package org.apache.flink.api.java.typeutils.runtime;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
 import org.apache.flink.api.common.typeutils.GenericTypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.util.InstantiationUtil;
@@ -167,13 +167,13 @@ public boolean canEqual(Object obj) {
 	}
 
 	@Override
-	public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	public TypeSerializerSchemaCompatibility<T> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		if (configSnapshot instanceof WritableSerializerConfigSnapshot
-				&& typeClass.equals(((WritableSerializerConfigSnapshot) configSnapshot).getTypeClass())) {
+				&& typeClass.equals(((WritableSerializerConfigSnapshot<?>) configSnapshot).getTypeClass())) {
 
-			return CompatibilityResult.compatible();
+			return TypeSerializerSchemaCompatibility.compatibleAsIs();
 		} else {
-			return CompatibilityResult.requiresMigration();
+			return TypeSerializerSchemaCompatibility.incompatible();
 		}
 	}
 
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/BackwardsCompatibleConfigSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/BackwardsCompatibleConfigSnapshot.java
new file mode 100644
index 00000000000..8bd2c97d341
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/BackwardsCompatibleConfigSnapshot.java
@@ -0,0 +1,103 @@
+/*
+ * 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.api.common.typeutils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+
+/**
+ * A utility {@link TypeSerializerConfigSnapshot} that is used for backwards compatibility purposes.
+ *
+ * @param <T> the data type that the wrapped serializer instance serializes.
+ */
+@Internal
+public class BackwardsCompatibleConfigSnapshot<T> extends TypeSerializerConfigSnapshot<T> {
+
+	private TypeSerializerConfigSnapshot<?> wrappedConfigSnapshot;
+
+	private TypeSerializer<T> serializerInstance;
+
+	public BackwardsCompatibleConfigSnapshot(
+			TypeSerializerConfigSnapshot<?> wrappedConfigSnapshot,
+			TypeSerializer<T> serializerInstance) {
+
+		this.wrappedConfigSnapshot = Preconditions.checkNotNull(wrappedConfigSnapshot);
+		this.serializerInstance = Preconditions.checkNotNull(serializerInstance);
+	}
+
+	@Override
+	public void write(DataOutputView out) throws IOException {
+		throw new UnsupportedOperationException(
+			"This is a dummy config snapshot used only for backwards compatibility.");
+	}
+
+	@Override
+	public void read(DataInputView in) throws IOException {
+		throw new UnsupportedOperationException(
+			"This is a dummy config snapshot used only for backwards compatibility.");
+	}
+
+	@Override
+	public int getVersion() {
+		throw new UnsupportedOperationException(
+			"This is a dummy config snapshot used only for backwards compatibility.");
+	}
+
+	@Override
+	public TypeSerializer<T> restoreSerializer() {
+		return serializerInstance;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	public TypeSerializerSchemaCompatibility<T> resolveSchemaCompatibility(TypeSerializer<?> newSerializer) {
+		return (TypeSerializerSchemaCompatibility<T>) wrappedConfigSnapshot.resolveSchemaCompatibility(newSerializer);
+	}
+
+	public TypeSerializerConfigSnapshot<?> getWrappedConfigSnapshot() {
+		return wrappedConfigSnapshot;
+	}
+
+	@Override
+	public int hashCode() {
+		int result = wrappedConfigSnapshot.hashCode();
+		result = 31 * result + serializerInstance.hashCode();
+		return result;
+	}
+
+	@Override
+	public boolean equals(Object o) {
+		if (this == o) {
+			return true;
+		}
+
+		if (o == null || getClass() != o.getClass()) {
+			return false;
+		}
+
+		BackwardsCompatibleConfigSnapshot<?> that = (BackwardsCompatibleConfigSnapshot<?>) o;
+
+		return that.wrappedConfigSnapshot.equals(wrappedConfigSnapshot)
+			&& that.serializerInstance.equals(serializerInstance);
+	}
+}
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompatibilityResult.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompatibilityResult.java
deleted file mode 100644
index 1e05d57854b..00000000000
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompatibilityResult.java
+++ /dev/null
@@ -1,116 +0,0 @@
-/*
- * 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.api.common.typeutils;
-
-import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.util.Preconditions;
-
-import javax.annotation.Nonnull;
-
-/**
- * A {@code CompatibilityResult} contains information about whether or not data migration
- * is required in order to continue using new serializers for previously serialized data.
- *
- * @param <T> the type of the data being migrated.
- */
-@PublicEvolving
-public final class CompatibilityResult<T> {
-
-	/** Whether or not migration is required. */
-	private final boolean requiresMigration;
-
-	/**
-	 * The convert deserializer to use for reading previous data during migration,
-	 * in the case that the preceding serializer cannot be found.
-	 *
-	 * <p>This is only relevant if migration is required.
-	 */
-	private final TypeDeserializer<T> convertDeserializer;
-
-	/**
-	 * Returns a result that signals that the new serializer is compatible and no migration is required.
-	 *
-	 * @return a result that signals migration is not required for the new serializer
-	 */
-	public static <T> CompatibilityResult<T> compatible() {
-		return new CompatibilityResult<>(false, null);
-	}
-
-	/**
-	 * Returns a result that signals migration to be performed, and in the case that the preceding serializer
-	 * cannot be found or restored to read the previous data during migration, a provided convert deserializer
-	 * can be used.
-	 *
-	 * @param convertDeserializer the convert deserializer to use, in the case that the preceding serializer
-	 *                            cannot be found.
-	 *
-	 * @param <T> the type of the data being migrated.
-	 *
-	 * @return a result that signals migration is necessary, also providing a convert deserializer.
-	 */
-	public static <T> CompatibilityResult<T> requiresMigration(@Nonnull TypeDeserializer<T> convertDeserializer) {
-		Preconditions.checkNotNull(convertDeserializer, "Convert deserializer cannot be null.");
-
-		return new CompatibilityResult<>(true, convertDeserializer);
-	}
-
-	/**
-	 * Returns a result that signals migration to be performed, and in the case that the preceding serializer
-	 * cannot be found or restored to read the previous data during migration, a provided convert serializer
-	 * can be used. The provided serializer will only be used for deserialization.
-	 *
-	 * @param convertSerializer the convert serializer to use, in the case that the preceding serializer
-	 *                          cannot be found. The provided serializer will only be used for deserialization.
-	 *
-	 * @param <T> the type of the data being migrated.
-	 *
-	 * @return a result that signals migration is necessary, also providing a convert serializer.
-	 */
-	public static <T> CompatibilityResult<T> requiresMigration(@Nonnull TypeSerializer<T> convertSerializer) {
-		Preconditions.checkNotNull(convertSerializer, "Convert serializer cannot be null.");
-
-		return new CompatibilityResult<>(true, new TypeDeserializerAdapter<>(convertSerializer));
-	}
-
-	/**
-	 * Returns a result that signals migration to be performed. The migration will fail if the preceding
-	 * serializer for the previous data cannot be found.
-	 *
-	 * <p>You can also provide a convert deserializer using {@link #requiresMigration(TypeDeserializer)}
-	 * or {@link #requiresMigration(TypeSerializer)}, which will be used as a fallback resort in such cases.
-	 *
-	 * @return a result that signals migration is necessary, without providing a convert deserializer.
-	 */
-	public static <T> CompatibilityResult<T> requiresMigration() {
-		return new CompatibilityResult<>(true, null);
-	}
-
-	private CompatibilityResult(boolean requiresMigration, TypeDeserializer<T> convertDeserializer) {
-		this.requiresMigration = requiresMigration;
-		this.convertDeserializer = convertDeserializer;
-	}
-
-	public TypeDeserializer<T> getConvertDeserializer() {
-		return convertDeserializer;
-	}
-
-	public boolean isRequiresMigration() {
-		return requiresMigration;
-	}
-}
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompatibilityUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompatibilityUtil.java
index 6c8583c1968..34cfc9bfa0f 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompatibilityUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompatibilityUtil.java
@@ -19,8 +19,6 @@
 
 import org.apache.flink.annotation.Internal;
 
-import javax.annotation.Nullable;
-
 /**
  * Utilities related to serializer compatibility.
  */
@@ -44,8 +42,6 @@
 	 *      If yes, use that for state migration and simply return the result.
 	 *   6. If all of above fails, state migration is required but could not be performed; throw exception.
 	 *
-	 * @param precedingSerializer the preceding serializer used to write the data, null if none could be retrieved
-	 * @param dummySerializerClassTag any class tags that identifies the preceding serializer as a dummy placeholder
 	 * @param precedingSerializerConfigSnapshot configuration snapshot of the preceding serializer
 	 * @param newSerializer the new serializer to ensure compatibility with
 	 *
@@ -54,31 +50,16 @@
 	 * @return the final resolved compatibility result
 	 */
 	@SuppressWarnings("unchecked")
-	public static <T> CompatibilityResult<T> resolveCompatibilityResult(
-			@Nullable TypeSerializer<?> precedingSerializer,
-			Class<?> dummySerializerClassTag,
-			TypeSerializerConfigSnapshot precedingSerializerConfigSnapshot,
-			TypeSerializer<T> newSerializer) {
+	public static <T> TypeSerializerSchemaCompatibility<T> resolveCompatibilityResult(
+			TypeSerializerConfigSnapshot<T> precedingSerializerConfigSnapshot,
+			TypeSerializer<?> newSerializer) {
 
 		if (precedingSerializerConfigSnapshot != null) {
-			CompatibilityResult<T> initialResult = newSerializer.ensureCompatibility(precedingSerializerConfigSnapshot);
-
-			if (!initialResult.isRequiresMigration()) {
-				return initialResult;
-			} else {
-				if (precedingSerializer != null && !(precedingSerializer.getClass().equals(dummySerializerClassTag))) {
-					// if the preceding serializer exists and is not a dummy, use
-					// that for converting instead of any provided convert deserializer
-					return CompatibilityResult.requiresMigration((TypeSerializer<T>) precedingSerializer);
-				} else {
-					// requires migration (may or may not have a convert deserializer)
-					return initialResult;
-				}
-			}
+			return precedingSerializerConfigSnapshot.resolveSchemaCompatibility(newSerializer);
 		} else {
 			// if the configuration snapshot of the preceding serializer cannot be provided,
 			// we can only simply assume that the new serializer is compatible
-			return CompatibilityResult.compatible();
+			return TypeSerializerSchemaCompatibility.compatibleAsIs();
 		}
 	}
 
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializer.java
new file mode 100644
index 00000000000..c68bec7141a
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializer.java
@@ -0,0 +1,105 @@
+/*
+ * 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.api.common.typeutils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.util.Preconditions;
+
+/**
+ * A base class for simple composite type serializers which consists of multiple nested
+ * type serializers. The constructor takes a pre-instantiated {@link CompositeTypeSerializerConfigSnapshot}
+ * instance, which is cached and returned on each {@link #snapshotConfiguration()} call.
+ * When compatibility is to be ensured with a restored configuration snapshot, that snapshot is checked whether or not
+ * it is of the same class as the cached one. The compatibility check is performed by simply ensuring that
+ * all nested serializers are compatible with the corresponding nested configuration snapshots.
+ */
+@Internal
+public abstract class CompositeTypeSerializer<T> extends TypeSerializer<T> {
+
+	private static final long serialVersionUID = -1460007327926564185L;
+
+	private final CompositeTypeSerializerConfigSnapshot<T> cachedConfigSnapshot;
+
+	private final TypeSerializer<?>[] nestedSerializers;
+
+	/**
+	 * Super constructor for composite type serializers.
+	 *
+	 * @param cachedConfigSnapshot the {@link CompositeTypeSerializerConfigSnapshot} to be cached.
+	 * @param nestedSerializers the nested serializers captured by the cached config snapshot, in the exact same order.
+	 */
+	public CompositeTypeSerializer(
+			CompositeTypeSerializerConfigSnapshot<T> cachedConfigSnapshot,
+			TypeSerializer<?>... nestedSerializers) {
+
+		this.cachedConfigSnapshot = Preconditions.checkNotNull(cachedConfigSnapshot);
+		this.nestedSerializers = Preconditions.checkNotNull(nestedSerializers);
+
+		Preconditions.checkArgument(
+			cachedConfigSnapshot.getNumNestedSerializers() == nestedSerializers.length,
+			"The cached composite type serializer configuration snapshot captures a different number of" +
+				" nested serializers than the length of the provided list of nested serializers.");
+	}
+
+	@Override
+	public final TypeSerializerConfigSnapshot<T> snapshotConfiguration() {
+		return cachedConfigSnapshot;
+	}
+
+	@Override
+	public TypeSerializerSchemaCompatibility<T> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
+		if (isComparableSnapshot(configSnapshot)) {
+			@SuppressWarnings("unchecked")
+			CompositeTypeSerializerConfigSnapshot<T> snapshot = (CompositeTypeSerializerConfigSnapshot<T>) configSnapshot;
+
+			int i = 0;
+			for (TypeSerializer<?> nestedSerializer : nestedSerializers) {
+				TypeSerializerSchemaCompatibility<?> compatibilityResult = CompatibilityUtil.resolveCompatibilityResult(
+					snapshot.getNestedSerializerConfigSnapshot(i),
+					nestedSerializer);
+
+				if (compatibilityResult.isIncompatible()) {
+					return TypeSerializerSchemaCompatibility.incompatible();
+				}
+
+				i++;
+			}
+
+			return TypeSerializerSchemaCompatibility.compatibleAsIs();
+		} else {
+			return TypeSerializerSchemaCompatibility.incompatible();
+		}
+	}
+
+	public TypeSerializer<?>[] getNestedSerializers() {
+		return nestedSerializers;
+	}
+
+	/**
+	 * Subclasses can override this if the serializer recognizes configuration snapshot
+	 * classes beyond the cached one. For example, composite type serializers which in previous versions
+	 * return a different configuration snapshot class than the one currently used could override this
+	 * method for backwards compatibility.
+	 *
+	 * @param configSnapshot the config snapshot to compare with.
+	 */
+	protected boolean isComparableSnapshot(TypeSerializerConfigSnapshot<?> configSnapshot) {
+		return configSnapshot.getClass().equals(cachedConfigSnapshot.getClass());
+	}
+}
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializerConfigSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializerConfigSnapshot.java
index 45b78c1a1f0..7005d6cf06e 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializerConfigSnapshot.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializerConfigSnapshot.java
@@ -38,9 +38,9 @@
  * is required.
  */
 @Internal
-public abstract class CompositeTypeSerializerConfigSnapshot extends TypeSerializerConfigSnapshot {
+public abstract class CompositeTypeSerializerConfigSnapshot<T> extends TypeSerializerConfigSnapshot<T> {
 
-	private List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> nestedSerializersAndConfigs;
+	private List<TypeSerializerConfigSnapshot<?>> nestedSerializerConfigs;
 
 	/** This empty nullary constructor is required for deserializing the configuration. */
 	public CompositeTypeSerializerConfigSnapshot() {}
@@ -48,35 +48,56 @@ public CompositeTypeSerializerConfigSnapshot() {}
 	public CompositeTypeSerializerConfigSnapshot(TypeSerializer<?>... nestedSerializers) {
 		Preconditions.checkNotNull(nestedSerializers);
 
-		this.nestedSerializersAndConfigs = new ArrayList<>(nestedSerializers.length);
+		this.nestedSerializerConfigs = new ArrayList<>(nestedSerializers.length);
 		for (TypeSerializer<?> nestedSerializer : nestedSerializers) {
-			TypeSerializerConfigSnapshot configSnapshot = nestedSerializer.snapshotConfiguration();
-			this.nestedSerializersAndConfigs.add(
-				new Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>(
-					nestedSerializer.duplicate(),
-					Preconditions.checkNotNull(configSnapshot)));
+			this.nestedSerializerConfigs.add(
+				Preconditions.checkNotNull(
+					nestedSerializer.snapshotConfiguration(),
+					"Configuration snapshots of nested serializers cannot be null."));
 		}
 	}
 
 	@Override
 	public void write(DataOutputView out) throws IOException {
 		super.write(out);
-		TypeSerializerSerializationUtil.writeSerializersAndConfigsWithResilience(out, nestedSerializersAndConfigs);
+		TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshots(out, nestedSerializerConfigs);
 	}
 
 	@Override
 	public void read(DataInputView in) throws IOException {
 		super.read(in);
-		this.nestedSerializersAndConfigs =
-			TypeSerializerSerializationUtil.readSerializersAndConfigsWithResilience(in, getUserCodeClassLoader());
+
+		if (!containsSerializers()) {
+			this.nestedSerializerConfigs = TypeSerializerConfigSnapshotSerializationUtil
+				.readSerializerConfigSnapshots(in, getUserCodeClassLoader());
+		} else {
+			// backwards compatible path
+			List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> nestedSerializersAndConfigs =
+				TypeSerializerSerializationUtil.readSerializersAndConfigsWithResilience(in, getUserCodeClassLoader());
+
+			this.nestedSerializerConfigs = new ArrayList<>(nestedSerializersAndConfigs.size());
+			for (Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> entry : nestedSerializersAndConfigs) {
+				this.nestedSerializerConfigs.add(new BackwardsCompatibleConfigSnapshot<>(entry.f1, entry.f0));
+			}
+		}
+	}
+
+	/**
+	 * Return whether or not this composite type serializer config snapshot still contains
+	 * serializers. Subclasses should uptick their version, and use that to compare against the read version
+	 * of the config snapshot to determine this. By default, it is assumed that all composite
+	 * type serializer config snapshots do not contain serializers.
+	 */
+	protected boolean containsSerializers() {
+		return false;
 	}
 
-	public List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> getNestedSerializersAndConfigs() {
-		return nestedSerializersAndConfigs;
+	public int getNumNestedSerializers() {
+		return nestedSerializerConfigs.size();
 	}
 
-	public Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> getSingleNestedSerializerAndConfig() {
-		return nestedSerializersAndConfigs.get(0);
+	public TypeSerializerConfigSnapshot<?> getNestedSerializerConfigSnapshot(int index) {
+		return nestedSerializerConfigs.get(index);
 	}
 
 	@Override
@@ -90,11 +111,56 @@ public boolean equals(Object obj) {
 		}
 
 		return (obj.getClass().equals(getClass()))
-				&& nestedSerializersAndConfigs.equals(((CompositeTypeSerializerConfigSnapshot) obj).getNestedSerializersAndConfigs());
+				&& nestedSerializerConfigs.equals(((CompositeTypeSerializerConfigSnapshot) obj).nestedSerializerConfigs);
 	}
 
 	@Override
 	public int hashCode() {
-		return nestedSerializersAndConfigs.hashCode();
+		return nestedSerializerConfigs.hashCode();
+	}
+
+	@Override
+	public final TypeSerializer<T> restoreSerializer() {
+		TypeSerializer<?>[] restoredNestedSerializers = new TypeSerializer[nestedSerializerConfigs.size()];
+
+		int i = 0;
+		for (TypeSerializerConfigSnapshot<?> config : nestedSerializerConfigs) {
+			restoredNestedSerializers[i] = config.restoreSerializer();
+			i++;
+		}
+
+		return restoreSerializer(restoredNestedSerializers);
+	}
+
+	@Override
+	public TypeSerializerSchemaCompatibility<T> resolveSchemaCompatibility(TypeSerializer<?> newSerializer) {
+		if (isRecognizableSerializer(newSerializer)) {
+			TypeSerializer<?>[] newNestedSerializers = ((CompositeTypeSerializer) newSerializer).getNestedSerializers();
+
+			int i = 0;
+			for (TypeSerializerConfigSnapshot<?> nestedSerializerConfig : nestedSerializerConfigs) {
+				TypeSerializerSchemaCompatibility<?> nestedSchemaCompat =
+					nestedSerializerConfig.resolveSchemaCompatibility(newNestedSerializers[i]);
+
+				if (nestedSchemaCompat.isIncompatible()) {
+					return TypeSerializerSchemaCompatibility.incompatible();
+				}
+				i++;
+			}
+
+			return TypeSerializerSchemaCompatibility.compatibleAsIs();
+		} else {
+			return TypeSerializerSchemaCompatibility.incompatible();
+		}
 	}
+
+	/**
+	 * Restore the composite type serializer with the restored nested serializers.
+	 *
+	 * @param restoredNestedSerializers the restored nested serializers.
+	 * @return the restored composite type serializer.
+	 */
+	protected abstract TypeSerializer<T> restoreSerializer(TypeSerializer<?>... restoredNestedSerializers);
+
+	protected abstract boolean isRecognizableSerializer(TypeSerializer<?> newSerializer);
 }
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/GenericTypeSerializerConfigSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/GenericTypeSerializerConfigSnapshot.java
index 4edfe123241..ae95bfdda0c 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/GenericTypeSerializerConfigSnapshot.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/GenericTypeSerializerConfigSnapshot.java
@@ -31,7 +31,7 @@
  * @param <T> The type to be instantiated.
  */
 @Internal
-public abstract class GenericTypeSerializerConfigSnapshot<T> extends TypeSerializerConfigSnapshot {
+public abstract class GenericTypeSerializerConfigSnapshot<T> extends TypeSerializerConfigSnapshot<T> {
 
 	private Class<T> typeClass;
 
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/ParameterlessTypeSerializerConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/ParameterlessTypeSerializerConfig.java
index 7ba7dd452d6..6fc6d172491 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/ParameterlessTypeSerializerConfig.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/ParameterlessTypeSerializerConfig.java
@@ -29,7 +29,7 @@
  * A base class for {@link TypeSerializerConfigSnapshot}s that do not have any parameters.
  */
 @Internal
-public final class ParameterlessTypeSerializerConfig extends TypeSerializerConfigSnapshot {
+public final class ParameterlessTypeSerializerConfig<T> extends TypeSerializerConfigSnapshot<T> {
 
 	private static final int VERSION = 1;
 
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeDeserializerAdapter.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeDeserializerAdapter.java
index fb59602c5f5..abe7fdd2f85 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeDeserializerAdapter.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeDeserializerAdapter.java
@@ -130,12 +130,12 @@ public void copy(DataInputView source, DataOutputView target) throws IOException
 			"This is a TypeDeserializerAdapter used only for deserialization; this method should not be used.");
 	}
 
-	public TypeSerializerConfigSnapshot snapshotConfiguration() {
+	public TypeSerializerConfigSnapshot<T> snapshotConfiguration() {
 		throw new UnsupportedOperationException(
 			"This is a TypeDeserializerAdapter used only for deserialization; this method should not be used.");
 	}
 
-	public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	public TypeSerializerSchemaCompatibility<T> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		throw new UnsupportedOperationException(
 			"This is a TypeDeserializerAdapter used only for deserialization; this method should not be used.");
 	}
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
index a606a181d1d..369b2780c74 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializer.java
@@ -179,41 +179,44 @@
 	 *
 	 * @return snapshot of the serializer's current configuration (cannot be {@code null}).
 	 */
-	public abstract TypeSerializerConfigSnapshot snapshotConfiguration();
+	public abstract TypeSerializerConfigSnapshot<T> snapshotConfiguration();
 
 	/**
 	 * Ensure compatibility of this serializer with a preceding serializer that was registered for serialization of
 	 * the same managed state (if any - this method is only relevant if this serializer is registered for
 	 * serialization of managed state).
 	 *
-	 * The compatibility check in this method should be performed by inspecting the preceding serializer's configuration
+	 * <p>The compatibility check in this method should be performed by inspecting the preceding serializer's configuration
 	 * snapshot. The method may reconfigure the serializer (if required and possible) so that it may be compatible,
 	 * or provide a signaling result that informs Flink that state migration is necessary before continuing to use
 	 * this serializer.
 	 *
 	 * <p>The result can be one of the following:
 	 * <ul>
-	 *     <li>{@link CompatibilityResult#compatible()}: this signals Flink that this serializer is compatible, or
+	 *     <li>{@link TypeSerializerSchemaCompatibility#compatible()}: this signals Flink that this serializer is compatible, or
 	 *     has been reconfigured to be compatible, to continue reading previous data, and that the
 	 *     serialization schema remains the same. No migration needs to be performed.</li>
 	 *
-	 *     <li>{@link CompatibilityResult#requiresMigration(TypeDeserializer)}: this signals Flink that
+	 *     <li>{@link TypeSerializerSchemaCompatibility#requiresMigration(TypeDeserializer)}: this signals Flink that
 	 *     migration needs to be performed, because this serializer is not compatible, or cannot be reconfigured to be
 	 *     compatible, for previous data. Furthermore, in the case that the preceding serializer cannot be found or
 	 *     restored to read the previous data to perform the migration, the provided convert deserializer can be
 	 *     used as a fallback resort.</li>
 	 *
-	 *     <li>{@link CompatibilityResult#requiresMigration()}: this signals Flink that migration needs to be
+	 *     <li>{@link TypeSerializerSchemaCompatibility#requiresMigration()}: this signals Flink that migration needs to be
 	 *     performed, because this serializer is not compatible, or cannot be reconfigured to be compatible, for
 	 *     previous data. If the preceding serializer cannot be found (either its implementation changed or it was
 	 *     removed from the classpath) then the migration will fail due to incapability to read previous data.</li>
 	 * </ul>
 	 *
-	 * @see CompatibilityResult
+	 * @see TypeSerializerSchemaCompatibility
 	 *
 	 * @param configSnapshot configuration snapshot of a preceding serializer for the same managed state
 	 *
 	 * @return the determined compatibility result (cannot be {@code null}).
+	 *
+	 * @deprecated // TODO remove this method in follow-up commits
 	 */
-	public abstract CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot);
+	@Deprecated
+	public abstract TypeSerializerSchemaCompatibility<T> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot);
 }
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshot.java
index 389d141fff0..489855f64c2 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshot.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshot.java
@@ -25,12 +25,33 @@
 
 /**
  * A {@code TypeSerializerConfigSnapshot} is a point-in-time view of a {@link TypeSerializer's} configuration.
- * The configuration snapshot of a serializer is persisted along with checkpoints of the managed state that the
- * serializer is registered to.
+ * The configuration snapshot of a serializer is persisted within checkpoints
+ * as a single source of meta information about the schema of serialized data in the checkpoint.
+ * This serves three purposes:
  *
- * <p>The persisted configuration may later on be used by new serializers to ensure serialization compatibility
- * for the same managed state. In order for new serializers to be able to ensure this, the configuration snapshot
- * should encode sufficient information about:
+ * <ul>
+ *   <li><strong>Capturing serializer parameters and schema:</strong> a serializer's configuration snapshot
+ *   represents information about the parameters, state, and schema of a serializer.
+ *   This is explained in more detail below.</li>
+ *
+ *   <li><strong>Compatibility checks for new serializers:</strong> when new serializers are available,
+ *   they need to be checked whether or not they are compatible to read the data written by the previous serializer.
+ *   This is performed by providing the serializer configuration snapshots in checkpoints to the corresponding
+ *   new serializers.</li>
+ *
+ *   <li><strong>Factory for a read serializer when schema conversion is required:<strong> in the case that new
+ *   serializers are not compatible to read previous data, a schema conversion process executed across all data
+ *   is required before the new serializer can be continued to be used. This conversion process requires a compatible
+ *   read serializer to restore serialized bytes as objects, and then written back again using the new serializer.
+ *   In this scenario, the serializer configuration snapshots in checkpoints doubles as a factory for the read
+ *   serializer of the conversion process.</li>
+ * </ul>
+ *
+ * <h2>Serializer Configuration and Schema</h2>
+ *
+ * <p>Since serializer configuration snapshots needs to be used to ensure serialization compatibility
+ * for the same managed state as well as serving as a factory for compatible read serializers, the configuration
+ * snapshot should encode sufficient information about:
  *
  * <ul>
  *   <li><strong>Parameter settings of the serializer:</strong> parameters of the serializer include settings
@@ -38,18 +59,50 @@
  *   has nested serializers, then the configuration snapshot should also contain the parameters of the nested
  *   serializers.</li>
  *
- *   <li><strong>Serialization schema of the serializer:</strong> the data format used by the serializer.</li>
+ *   <li><strong>Serialization schema of the serializer:</strong> the binary format used by the serializer, or
+ *   in other words, the schema of data written by the serializer.</li>
  * </ul>
  *
  * <p>NOTE: Implementations must contain the default empty nullary constructor. This is required to be able to
  * deserialize the configuration snapshot from its binary form.
+ *
+ * @param <T> The data type that the originating serializer of this configuration serializes.
  */
 @PublicEvolving
-public abstract class TypeSerializerConfigSnapshot extends VersionedIOReadableWritable {
+public abstract class TypeSerializerConfigSnapshot<T> extends VersionedIOReadableWritable {
 
 	/** The user code class loader; only relevant if this configuration instance was deserialized from binary form. */
 	private ClassLoader userCodeClassLoader;
 
+	/**
+	 * Creates a serializer using this configuration, that is capable of reading data
+	 * written by the serializer described by this configuration.
+	 *
+	 * @return the restored serializer.
+	 */
+	public TypeSerializer<T> restoreSerializer() {
+		// TODO this method actually should not have a default implementation;
+		// TODO this placeholder should be removed as soon as all subclasses have a proper implementation in place, and
+		// TODO the method is properly integrated in state backends' restore procedures
+		throw new UnsupportedOperationException();
+	}
+
+	/**
+	 * Determines the serialization schema compatibility between a new serializer
+	 * and the original serializer described by this configuration.
+	 *
+	 * @param newSerializer the new serializer to determine serialization schema compatibility with.
+	 *
+	 * @return the schema compatibility result.
+	 */
+	public TypeSerializerSchemaCompatibility<T> resolveSchemaCompatibility(TypeSerializer<?> newSerializer) {
+		// TODO this method actually should not have a default implementation;
+		// TODO this placeholder should be removed as soon as all subclasses have a proper implementation in place, and
+		// TODO the method is properly integrated in state backends' restore procedures
+//		throw new UnsupportedOperationException();
+		return TypeSerializerSchemaCompatibility.compatibleAsIs();
+	}
+
 	/**
 	 * Set the user code class loader.
 	 * Only relevant if this configuration instance was deserialized from binary form.
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotSerializationUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotSerializationUtil.java
new file mode 100644
index 00000000000..d7d4e16e45c
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotSerializationUtil.java
@@ -0,0 +1,185 @@
+/*
+ * 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.api.common.typeutils;
+
+import org.apache.flink.core.io.VersionedIOReadableWritable;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.util.InstantiationUtil;
+import org.apache.flink.util.Preconditions;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Utility methods for serialization of {@link TypeSerializerConfigSnapshot}.
+ */
+public class TypeSerializerConfigSnapshotSerializationUtil {
+
+	/**
+	 * Writes a {@link TypeSerializerConfigSnapshot} to the provided data output view.
+	 *
+	 * <p>It is written with a format that can be later read again using
+	 * {@link #readSerializerConfigSnapshot(DataInputView, ClassLoader)}.
+	 *
+	 * @param out the data output view
+	 * @param serializerConfigSnapshot the serializer configuration snapshot to write
+	 *
+	 * @throws IOException
+	 */
+	public static void writeSerializerConfigSnapshot(
+			DataOutputView out,
+			TypeSerializerConfigSnapshot serializerConfigSnapshot) throws IOException {
+
+		new TypeSerializerConfigSnapshotSerializationProxy(serializerConfigSnapshot).write(out);
+	}
+
+	/**
+	 * Reads from a data input view a {@link TypeSerializerConfigSnapshot} that was previously
+	 * written using {@link TypeSerializerConfigSnapshotSerializationUtil#writeSerializerConfigSnapshot(DataOutputView, TypeSerializerConfigSnapshot)}.
+	 *
+	 * @param in the data input view
+	 * @param userCodeClassLoader the user code class loader to use
+	 *
+	 * @return the read serializer configuration snapshot
+	 *
+	 * @throws IOException
+	 */
+	public static TypeSerializerConfigSnapshot readSerializerConfigSnapshot(
+			DataInputView in,
+			ClassLoader userCodeClassLoader) throws IOException {
+
+		final TypeSerializerConfigSnapshotSerializationProxy proxy = new TypeSerializerConfigSnapshotSerializationProxy(userCodeClassLoader);
+		proxy.read(in);
+
+		return proxy.getSerializerConfigSnapshot();
+	}
+
+	/**
+	 * Writes multiple {@link TypeSerializerConfigSnapshot}s to the provided data output view.
+	 *
+	 * <p>It is written with a format that can be later read again using
+	 * {@link #readSerializerConfigSnapshots(DataInputView, ClassLoader)}.
+	 *
+	 * @param out the data output view
+	 * @param serializerConfigSnapshots the serializer configuration snapshots to write
+	 *
+	 * @throws IOException
+	 */
+	public static void writeSerializerConfigSnapshots(
+			DataOutputView out,
+			List<TypeSerializerConfigSnapshot<?>> serializerConfigSnapshots) throws IOException {
+
+		out.writeInt(serializerConfigSnapshots.size());
+
+		for (TypeSerializerConfigSnapshot<?> snapshot : serializerConfigSnapshots) {
+			new TypeSerializerConfigSnapshotSerializationProxy(snapshot).write(out);
+		}
+	}
+
+	/**
+	 * Reads from a data input view multiple {@link TypeSerializerConfigSnapshot}s that was previously
+	 * written using {@link TypeSerializerConfigSnapshotSerializationUtil#writeSerializerConfigSnapshot(DataOutputView, TypeSerializerConfigSnapshot)}.
+	 *
+	 * @param in the data input view
+	 * @param userCodeClassLoader the user code class loader to use
+	 *
+	 * @return the read serializer configuration snapshots
+	 *
+	 * @throws IOException
+	 */
+	public static List<TypeSerializerConfigSnapshot<?>> readSerializerConfigSnapshots(
+			DataInputView in,
+			ClassLoader userCodeClassLoader) throws IOException {
+
+		int numFields = in.readInt();
+		final List<TypeSerializerConfigSnapshot<?>> serializerConfigSnapshots = new ArrayList<>(numFields);
+
+		TypeSerializerConfigSnapshotSerializationProxy proxy;
+		for (int i = 0; i < numFields; i++) {
+			proxy = new TypeSerializerConfigSnapshotSerializationProxy(userCodeClassLoader);
+			proxy.read(in);
+			serializerConfigSnapshots.add(proxy.getSerializerConfigSnapshot());
+		}
+
+		return serializerConfigSnapshots;
+	}
+
+	/**
+	 * Utility serialization proxy for a {@link TypeSerializerConfigSnapshot}.
+	 */
+	static final class TypeSerializerConfigSnapshotSerializationProxy extends VersionedIOReadableWritable {
+
+		private static final int VERSION = 1;
+
+		private ClassLoader userCodeClassLoader;
+		private TypeSerializerConfigSnapshot<?> serializerConfigSnapshot;
+
+		TypeSerializerConfigSnapshotSerializationProxy(ClassLoader userCodeClassLoader) {
+			this.userCodeClassLoader = Preconditions.checkNotNull(userCodeClassLoader);
+		}
+
+		TypeSerializerConfigSnapshotSerializationProxy(TypeSerializerConfigSnapshot<?> serializerConfigSnapshot) {
+			this.serializerConfigSnapshot = serializerConfigSnapshot;
+		}
+
+		@Override
+		public void write(DataOutputView out) throws IOException {
+			super.write(out);
+
+			// config snapshot class, so that we can re-instantiate the
+			// correct type of config snapshot instance when deserializing
+			out.writeUTF(serializerConfigSnapshot.getClass().getName());
+
+			// the actual configuration parameters
+			serializerConfigSnapshot.write(out);
+		}
+
+		@SuppressWarnings("unchecked")
+		@Override
+		public void read(DataInputView in) throws IOException {
+			super.read(in);
+
+			String serializerConfigClassname = in.readUTF();
+			Class<? extends TypeSerializerConfigSnapshot> serializerConfigSnapshotClass;
+			try {
+				serializerConfigSnapshotClass = (Class<? extends TypeSerializerConfigSnapshot>)
+					Class.forName(serializerConfigClassname, true, userCodeClassLoader);
+			} catch (ClassNotFoundException e) {
+				throw new IOException(
+					"Could not find requested TypeSerializerConfigSnapshot class "
+						+ serializerConfigClassname +  " in classpath.", e);
+			}
+
+			serializerConfigSnapshot = InstantiationUtil.instantiate(serializerConfigSnapshotClass);
+			serializerConfigSnapshot.setUserCodeClassLoader(userCodeClassLoader);
+			serializerConfigSnapshot.read(in);
+		}
+
+		@Override
+		public int getVersion() {
+			return VERSION;
+		}
+
+		TypeSerializerConfigSnapshot<?> getSerializerConfigSnapshot() {
+			return serializerConfigSnapshot;
+		}
+	}
+}
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSchemaCompatibility.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSchemaCompatibility.java
new file mode 100644
index 00000000000..585d3c60638
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSchemaCompatibility.java
@@ -0,0 +1,93 @@
+/*
+ * 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.api.common.typeutils;
+
+import org.apache.flink.annotation.PublicEvolving;
+import org.apache.flink.util.Preconditions;
+
+import javax.annotation.Nullable;
+
+/**
+ * A {@code TypeSerializerSchemaCompatibility} contains information about whether or not data migration
+ * is required in order to continue using new serializers for previously serialized data.
+ *
+ * @param <T> the type of the data being migrated.
+ */
+@PublicEvolving
+public final class TypeSerializerSchemaCompatibility<T> {
+
+	enum Type {
+		COMPATIBLE_AS_IS,
+		COMPATIBLE_AFTER_RECONFIGURATION,
+		INCOMPATIBLE
+	}
+
+	/** Whether or not migration is required. */
+	private final Type resultType;
+
+	/** */
+	private final TypeSerializer<T> reconfiguredNewSerializer;
+
+	/**
+	 * Returns a result that signals that the new serializer is compatible and no migration is required.
+	 *
+	 * @return a result that signals migration is not required for the new serializer.
+	 */
+	public static <T> TypeSerializerSchemaCompatibility<T> compatibleAsIs() {
+		return new TypeSerializerSchemaCompatibility<>(Type.COMPATIBLE_AS_IS, null);
+	}
+
+	public static <T> TypeSerializerSchemaCompatibility<T> compatibleAfterReconfiguration(TypeSerializer<T> reconfiguredSerializer) {
+		return new TypeSerializerSchemaCompatibility<>(Type.COMPATIBLE_AFTER_RECONFIGURATION, reconfiguredSerializer);
+	}
+
+	/**
+	 * Returns a result that signals a full-pass state conversion needs to be performed.
+	 *
+	 * @return a result that signals a full-pass state conversion is necessary.
+	 */
+	public static <T> TypeSerializerSchemaCompatibility<T> incompatible() {
+		return new TypeSerializerSchemaCompatibility<T>(Type.INCOMPATIBLE, null);
+	}
+
+	private TypeSerializerSchemaCompatibility(Type resultType, @Nullable TypeSerializer<T> reconfiguredNewSerializer) {
+		this.resultType = Preconditions.checkNotNull(resultType);
+
+		if (resultType == Type.COMPATIBLE_AFTER_RECONFIGURATION && reconfiguredNewSerializer == null) {
+			throw new IllegalArgumentException();
+		}
+		this.reconfiguredNewSerializer = reconfiguredNewSerializer;
+	}
+
+	public boolean isCompatibleAsIs() {
+		return resultType == Type.COMPATIBLE_AS_IS;
+	}
+
+	public boolean isCompatibleAfterReconfiguration() {
+		return resultType == Type.COMPATIBLE_AFTER_RECONFIGURATION;
+	}
+
+	public boolean isIncompatible() {
+		return resultType == Type.INCOMPATIBLE;
+	}
+
+	public TypeSerializer<T> getReconfiguredNewSerializer() {
+		return reconfiguredNewSerializer;
+	}
+}
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtil.java
index e83b8c71ec3..d02a02c0ac6 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtil.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtil.java
@@ -40,9 +40,14 @@
 import java.util.List;
 
 /**
- * Utility methods for serialization of {@link TypeSerializer} and {@link TypeSerializerConfigSnapshot}.
+ * Utility methods for serialization of {@link TypeSerializer}.
+ *
+ * @deprecated This utility class was used to write serializers into checkpoints.
+ *             Starting from Flink 1.6.x, this should no longer happen, and therefore
+ *             this class is deprecated. It remains here for backwards compatibility paths.
  */
 @Internal
+@Deprecated
 public class TypeSerializerSerializationUtil {
 
 	private static final Logger LOG = LoggerFactory.getLogger(TypeSerializerSerializationUtil.class);
@@ -154,7 +159,7 @@ public static void writeSerializersAndConfigsWithResilience(
 				writeSerializer(bufferWrapper, serAndConfSnapshot.f0);
 
 				out.writeInt(bufferWithPos.getPosition());
-				writeSerializerConfigSnapshot(bufferWrapper, serAndConfSnapshot.f1);
+				TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(bufferWrapper, serAndConfSnapshot.f1);
 			}
 
 			out.writeInt(bufferWithPos.getPosition());
@@ -197,7 +202,7 @@ public static void writeSerializersAndConfigsWithResilience(
 			new ArrayList<>(numSerializersAndConfigSnapshots);
 
 		TypeSerializer<?> serializer;
-		TypeSerializerConfigSnapshot configSnapshot;
+		TypeSerializerConfigSnapshot<?> configSnapshot;
 		try (
 			ByteArrayInputStreamWithPos bufferWithPos = new ByteArrayInputStreamWithPos(buffer);
 			DataInputViewStreamWrapper bufferWrapper = new DataInputViewStreamWrapper(bufferWithPos)) {
@@ -208,105 +213,23 @@ public static void writeSerializersAndConfigsWithResilience(
 				serializer = tryReadSerializer(bufferWrapper, userCodeClassLoader, true);
 
 				bufferWithPos.setPosition(offsets[i * 2 + 1]);
-				configSnapshot = readSerializerConfigSnapshot(bufferWrapper, userCodeClassLoader);
 
-				serializersAndConfigSnapshots.add(
-					new Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>(serializer, configSnapshot));
+				// the config snapshot is replaced with a dummy one, which wraps
+				// the actual config snapshot and the deserialized serializer.
+				// this is for backwards compatibility reasons, since before Flink 1.6, some serializers
+				// do not return config snapshots that can be used as a factory for themselves.
+				configSnapshot = new BackwardsCompatibleConfigSnapshot<>(
+					TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(
+						bufferWrapper, userCodeClassLoader),
+					serializer);
+
+				serializersAndConfigSnapshots.add(new Tuple2<>(serializer, configSnapshot));
 			}
 		}
 
 		return serializersAndConfigSnapshots;
 	}
 
-	/**
-	 * Writes a {@link TypeSerializerConfigSnapshot} to the provided data output view.
-	 *
-	 * <p>It is written with a format that can be later read again using
-	 * {@link #readSerializerConfigSnapshot(DataInputView, ClassLoader)}.
-	 *
-	 * @param out the data output view
-	 * @param serializerConfigSnapshot the serializer configuration snapshot to write
-	 *
-	 * @throws IOException
-	 */
-	public static void writeSerializerConfigSnapshot(
-			DataOutputView out,
-			TypeSerializerConfigSnapshot serializerConfigSnapshot) throws IOException {
-
-		new TypeSerializerConfigSnapshotSerializationProxy(serializerConfigSnapshot).write(out);
-	}
-
-	/**
-	 * Reads from a data input view a {@link TypeSerializerConfigSnapshot} that was previously
-	 * written using {@link #writeSerializerConfigSnapshot(DataOutputView, TypeSerializerConfigSnapshot)}.
-	 *
-	 * @param in the data input view
-	 * @param userCodeClassLoader the user code class loader to use
-	 *
-	 * @return the read serializer configuration snapshot
-	 *
-	 * @throws IOException
-	 */
-	public static TypeSerializerConfigSnapshot readSerializerConfigSnapshot(
-			DataInputView in,
-			ClassLoader userCodeClassLoader) throws IOException {
-
-		final TypeSerializerConfigSnapshotSerializationProxy proxy = new TypeSerializerConfigSnapshotSerializationProxy(userCodeClassLoader);
-		proxy.read(in);
-
-		return proxy.getSerializerConfigSnapshot();
-	}
-
-	/**
-	 * Writes multiple {@link TypeSerializerConfigSnapshot}s to the provided data output view.
-	 *
-	 * <p>It is written with a format that can be later read again using
-	 * {@link #readSerializerConfigSnapshots(DataInputView, ClassLoader)}.
-	 *
-	 * @param out the data output view
-	 * @param serializerConfigSnapshots the serializer configuration snapshots to write
-	 *
-	 * @throws IOException
-	 */
-	public static void writeSerializerConfigSnapshots(
-			DataOutputView out,
-			TypeSerializerConfigSnapshot... serializerConfigSnapshots) throws IOException {
-
-		out.writeInt(serializerConfigSnapshots.length);
-
-		for (TypeSerializerConfigSnapshot snapshot : serializerConfigSnapshots) {
-			new TypeSerializerConfigSnapshotSerializationProxy(snapshot).write(out);
-		}
-	}
-
-	/**
-	 * Reads from a data input view multiple {@link TypeSerializerConfigSnapshot}s that was previously
-	 * written using {@link #writeSerializerConfigSnapshot(DataOutputView, TypeSerializerConfigSnapshot)}.
-	 *
-	 * @param in the data input view
-	 * @param userCodeClassLoader the user code class loader to use
-	 *
-	 * @return the read serializer configuration snapshots
-	 *
-	 * @throws IOException
-	 */
-	public static TypeSerializerConfigSnapshot[] readSerializerConfigSnapshots(
-			DataInputView in,
-			ClassLoader userCodeClassLoader) throws IOException {
-
-		int numFields = in.readInt();
-		final TypeSerializerConfigSnapshot[] serializerConfigSnapshots = new TypeSerializerConfigSnapshot[numFields];
-
-		TypeSerializerConfigSnapshotSerializationProxy proxy;
-		for (int i = 0; i < numFields; i++) {
-			proxy = new TypeSerializerConfigSnapshotSerializationProxy(userCodeClassLoader);
-			proxy.read(in);
-			serializerConfigSnapshots[i] = proxy.getSerializerConfigSnapshot();
-		}
-
-		return serializerConfigSnapshots;
-	}
-
 	// -----------------------------------------------------------------------------------------------------
 
 	/**
@@ -384,64 +307,4 @@ public int getVersion() {
 		}
 	}
 
-	/**
-	 * Utility serialization proxy for a {@link TypeSerializerConfigSnapshot}.
-	 */
-	static final class TypeSerializerConfigSnapshotSerializationProxy extends VersionedIOReadableWritable {
-
-		private static final int VERSION = 1;
-
-		private ClassLoader userCodeClassLoader;
-		private TypeSerializerConfigSnapshot serializerConfigSnapshot;
-
-		TypeSerializerConfigSnapshotSerializationProxy(ClassLoader userCodeClassLoader) {
-			this.userCodeClassLoader = Preconditions.checkNotNull(userCodeClassLoader);
-		}
-
-		TypeSerializerConfigSnapshotSerializationProxy(TypeSerializerConfigSnapshot serializerConfigSnapshot) {
-			this.serializerConfigSnapshot = serializerConfigSnapshot;
-		}
-
-		@Override
-		public void write(DataOutputView out) throws IOException {
-			super.write(out);
-
-			// config snapshot class, so that we can re-instantiate the
-			// correct type of config snapshot instance when deserializing
-			out.writeUTF(serializerConfigSnapshot.getClass().getName());
-
-			// the actual configuration parameters
-			serializerConfigSnapshot.write(out);
-		}
-
-		@SuppressWarnings("unchecked")
-		@Override
-		public void read(DataInputView in) throws IOException {
-			super.read(in);
-
-			String serializerConfigClassname = in.readUTF();
-			Class<? extends TypeSerializerConfigSnapshot> serializerConfigSnapshotClass;
-			try {
-				serializerConfigSnapshotClass = (Class<? extends TypeSerializerConfigSnapshot>)
-					Class.forName(serializerConfigClassname, true, userCodeClassLoader);
-			} catch (ClassNotFoundException e) {
-				throw new IOException(
-					"Could not find requested TypeSerializerConfigSnapshot class "
-						+ serializerConfigClassname +  " in classpath.", e);
-			}
-
-			serializerConfigSnapshot = InstantiationUtil.instantiate(serializerConfigSnapshotClass);
-			serializerConfigSnapshot.setUserCodeClassLoader(userCodeClassLoader);
-			serializerConfigSnapshot.read(in);
-		}
-
-		@Override
-		public int getVersion() {
-			return VERSION;
-		}
-
-		TypeSerializerConfigSnapshot getSerializerConfigSnapshot() {
-			return serializerConfigSnapshot;
-		}
-	}
 }
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/UnloadableDummyTypeSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/UnloadableDummyTypeSerializer.java
index ddfeab41c1d..56918c77804 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/UnloadableDummyTypeSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/UnloadableDummyTypeSerializer.java
@@ -94,12 +94,12 @@ public void copy(DataInputView source, DataOutputView target) throws IOException
 	}
 
 	@Override
-	public TypeSerializerConfigSnapshot snapshotConfiguration() {
+	public TypeSerializerConfigSnapshot<T> snapshotConfiguration() {
 		throw new UnsupportedOperationException("This object is a dummy TypeSerializer.");
 	}
 
 	@Override
-	public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	public TypeSerializerSchemaCompatibility<T> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		throw new UnsupportedOperationException("This object is a dummy TypeSerializer.");
 	}
 
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CollectionSerializerConfigSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CollectionSerializerConfigSnapshot.java
index 55729852ed1..9e0ee220908 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CollectionSerializerConfigSnapshot.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CollectionSerializerConfigSnapshot.java
@@ -22,15 +22,23 @@
 import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 
+import java.util.Collection;
+
 /**
  * Configuration snapshot of a serializer for collection types.
  *
  * @param <T> Type of the element.
+ *
+ * @deprecated this configuration snapshot is not capable of being a factory for all serializers that
+ *             previously write this as their configuration snapshot, and therefore deprecated. It is no
+ *             longer written by any of Flink's serializers, but is still here for backwards compatibility.
  */
 @Internal
-public final class CollectionSerializerConfigSnapshot<T> extends CompositeTypeSerializerConfigSnapshot {
+@Deprecated
+public final class CollectionSerializerConfigSnapshot<C extends Collection<T>, T>
+		extends CompositeTypeSerializerConfigSnapshot<C> {
 
-	private static final int VERSION = 1;
+	private static final int VERSION = 2;
 
 	/** This empty nullary constructor is required for deserializing the configuration. */
 	public CollectionSerializerConfigSnapshot() {}
@@ -39,8 +47,29 @@ public CollectionSerializerConfigSnapshot(TypeSerializer<T> elementSerializer) {
 		super(elementSerializer);
 	}
 
+	@Override
+	public int[] getCompatibleVersions() {
+		return new int[]{VERSION, 1};
+	}
+
+	@Override
+	protected boolean containsSerializers() {
+		// versions that still used this config snapshot always still wrote the serializers
+		return true;
+	}
+
 	@Override
 	public int getVersion() {
 		return VERSION;
 	}
+
+	@Override
+	protected TypeSerializer<C> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+		throw new UnsupportedOperationException();
+	}
+
+	@Override
+	protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+		return newSerializer instanceof ListSerializer;
+	}
 }
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/EnumSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/EnumSerializer.java
index c40fefc1694..94824b62d18 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/EnumSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/EnumSerializer.java
@@ -28,7 +28,7 @@
 
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.GenericTypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
@@ -178,7 +178,7 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE
 
 	@SuppressWarnings("unchecked")
 	@Override
-	public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	public TypeSerializerSchemaCompatibility<T> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		if (configSnapshot instanceof EnumSerializerConfigSnapshot) {
 			final EnumSerializerConfigSnapshot<T> config = (EnumSerializerConfigSnapshot<T>) configSnapshot;
 
@@ -201,13 +201,13 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE
 							rebuiltEnumConstantToOrdinalMap.put(enumConstant, i);
 						} catch (IllegalArgumentException e) {
 							// a previous enum constant no longer exists, and therefore requires migration
-							return CompatibilityResult.requiresMigration();
+							return TypeSerializerSchemaCompatibility.incompatible();
 						}
 					}
 				} else {
 					// some enum constants have been removed (because there are
 					// fewer constants now), and therefore requires migration
-					return CompatibilityResult.requiresMigration();
+					return TypeSerializerSchemaCompatibility.incompatible();
 				}
 
 				// if there are new enum constants, append them to the end
@@ -225,11 +225,11 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE
 				// if we reach here, we can simply reconfigure ourselves to be compatible
 				this.values = reorderedEnumConstants;
 				this.valueToOrdinal = rebuiltEnumConstantToOrdinalMap;
-				return CompatibilityResult.compatible();
+				return TypeSerializerSchemaCompatibility.compatibleAsIs();
 			}
 		}
 
-		return CompatibilityResult.requiresMigration();
+		return TypeSerializerSchemaCompatibility.incompatible();
 	}
 
 	/**
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializer.java
index cdfc964e15d..115d15e5190 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializer.java
@@ -22,13 +22,9 @@
 import java.lang.reflect.Array;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
+
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 
@@ -40,7 +36,7 @@
  * @param <C> The component type.
  */
 @Internal
-public final class GenericArraySerializer<C> extends TypeSerializer<C[]> {
+public final class GenericArraySerializer<C> extends CompositeTypeSerializer<C[]> {
 
 	private static final long serialVersionUID = 1L;
 
@@ -52,8 +48,13 @@
 	
 	
 	public GenericArraySerializer(Class<C> componentClass, TypeSerializer<C> componentSerializer) {
-		this.componentClass = checkNotNull(componentClass);
-		this.componentSerializer = checkNotNull(componentSerializer);
+
+		super(
+			new GenericArraySerializerConfigSnapshot<>(checkNotNull(componentClass), checkNotNull(componentSerializer)),
+			componentSerializer);
+
+		this.componentClass = componentClass;
+		this.componentSerializer = componentSerializer;
 	}
 
 	@Override
@@ -193,41 +194,7 @@ public String toString() {
 		return "Serializer " + componentClass.getName() + "[]";
 	}
 
-	// --------------------------------------------------------------------------------------------
-	// Serializer configuration snapshotting & compatibility
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public GenericArraySerializerConfigSnapshot snapshotConfiguration() {
-		return new GenericArraySerializerConfigSnapshot<>(componentClass, componentSerializer);
-	}
-
-	@Override
-	public CompatibilityResult<C[]> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-		if (configSnapshot instanceof GenericArraySerializerConfigSnapshot) {
-			final GenericArraySerializerConfigSnapshot config = (GenericArraySerializerConfigSnapshot) configSnapshot;
-
-			if (componentClass.equals(config.getComponentClass())) {
-				Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> previousComponentSerializerAndConfig =
-					config.getSingleNestedSerializerAndConfig();
-
-				CompatibilityResult<C> compatResult = CompatibilityUtil.resolveCompatibilityResult(
-						previousComponentSerializerAndConfig.f0,
-						UnloadableDummyTypeSerializer.class,
-						previousComponentSerializerAndConfig.f1,
-						componentSerializer);
-
-				if (!compatResult.isRequiresMigration()) {
-					return CompatibilityResult.compatible();
-				} else if (compatResult.getConvertDeserializer() != null) {
-					return CompatibilityResult.requiresMigration(
-						new GenericArraySerializer<>(
-							componentClass,
-							new TypeDeserializerAdapter<>(compatResult.getConvertDeserializer())));
-				}
-			}
-		}
-
-		return CompatibilityResult.requiresMigration();
+	Class<C> getComponentClass() {
+		return componentClass;
 	}
 }
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializerConfigSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializerConfigSnapshot.java
index 70e52106b48..422c14ed2a9 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializerConfigSnapshot.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializerConfigSnapshot.java
@@ -21,6 +21,7 @@
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream;
 import org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream;
 import org.apache.flink.core.memory.DataInputView;
@@ -36,9 +37,9 @@
  * @param <C> The component type.
  */
 @Internal
-public final class GenericArraySerializerConfigSnapshot<C> extends CompositeTypeSerializerConfigSnapshot {
+public final class GenericArraySerializerConfigSnapshot<C> extends CompositeTypeSerializerConfigSnapshot<C[]> {
 
-	private static final int VERSION = 1;
+	private static final int VERSION = 2;
 
 	private Class<C> componentClass;
 
@@ -79,6 +80,11 @@ public int getVersion() {
 		return VERSION;
 	}
 
+	@Override
+	public int[] getCompatibleVersions() {
+		return new int[]{VERSION, 1};
+	}
+
 	public Class<C> getComponentClass() {
 		return componentClass;
 	}
@@ -94,4 +100,33 @@ public boolean equals(Object obj) {
 	public int hashCode() {
 		return super.hashCode() * 31 + componentClass.hashCode();
 	}
+
+	@Override
+	protected boolean containsSerializers() {
+		return getReadVersion() < 2;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	protected TypeSerializer<C[]> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+		return new GenericArraySerializer<>(
+			componentClass,
+			(TypeSerializer<C>) restoredNestedSerializers[0]);
+	}
+
+	@Override
+	protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+		return newSerializer instanceof GenericArraySerializer;
+	}
+
+	@Override
+	public TypeSerializerSchemaCompatibility<C[]> resolveSchemaCompatibility(TypeSerializer<?> newSerializer) {
+		TypeSerializerSchemaCompatibility<C[]> result = super.resolveSchemaCompatibility(newSerializer);
+
+		if (result.isIncompatible()
+				|| !componentClass.equals(((GenericArraySerializer) newSerializer).getComponentClass())) {
+			return TypeSerializerSchemaCompatibility.incompatible();
+		}
+		return result;
+	}
 }
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java
index c2b935c82e1..e7ca7c0b41c 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java
@@ -19,13 +19,9 @@
 package org.apache.flink.api.common.typeutils.base;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 
@@ -45,7 +41,7 @@
  * @param <T> The type of element in the list.
  */
 @Internal
-public final class ListSerializer<T> extends TypeSerializer<List<T>> {
+public final class ListSerializer<T> extends CompositeTypeSerializer<List<T>> {
 
 	private static final long serialVersionUID = 1119562170939152304L;
 
@@ -58,7 +54,12 @@
 	 * @param elementSerializer The serializer for the elements of the list
 	 */
 	public ListSerializer(TypeSerializer<T> elementSerializer) {
-		this.elementSerializer = checkNotNull(elementSerializer);
+
+		super(
+			new ListSerializerConfigSnapshot<>(checkNotNull(elementSerializer)),
+			elementSerializer);
+
+		this.elementSerializer = elementSerializer;
 	}
 
 	// ------------------------------------------------------------------------
@@ -174,35 +175,11 @@ public int hashCode() {
 		return elementSerializer.hashCode();
 	}
 
-	// --------------------------------------------------------------------------------------------
-	// Serializer configuration snapshotting & compatibility
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public CollectionSerializerConfigSnapshot snapshotConfiguration() {
-		return new CollectionSerializerConfigSnapshot<>(elementSerializer);
-	}
-
 	@Override
-	public CompatibilityResult<List<T>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-		if (configSnapshot instanceof CollectionSerializerConfigSnapshot) {
-			Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> previousElemSerializerAndConfig =
-				((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig();
-
-			CompatibilityResult<T> compatResult = CompatibilityUtil.resolveCompatibilityResult(
-					previousElemSerializerAndConfig.f0,
-					UnloadableDummyTypeSerializer.class,
-					previousElemSerializerAndConfig.f1,
-					elementSerializer);
-
-			if (!compatResult.isRequiresMigration()) {
-				return CompatibilityResult.compatible();
-			} else if (compatResult.getConvertDeserializer() != null) {
-				return CompatibilityResult.requiresMigration(
-					new ListSerializer<>(new TypeDeserializerAdapter<>(compatResult.getConvertDeserializer())));
-			}
-		}
-
-		return CompatibilityResult.requiresMigration();
+	protected boolean isComparableSnapshot(TypeSerializerConfigSnapshot<?> configSnapshot) {
+		// previous versions of the ListSerializer still wrote the deprecated
+		// CollectionSerializerConfigSnapshot as the configuration snapshot
+		return configSnapshot instanceof ListSerializerConfigSnapshot
+			|| configSnapshot instanceof CollectionSerializerConfigSnapshot;
 	}
 }
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializerConfigSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializerConfigSnapshot.java
new file mode 100644
index 00000000000..fff8e7017f0
--- /dev/null
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializerConfigSnapshot.java
@@ -0,0 +1,56 @@
+/*
+ * 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.api.common.typeutils.base;
+
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+
+import java.util.List;
+
+/**
+ * A {@link TypeSerializerConfigSnapshot} for the {@link ListSerializer}.
+ */
+public class ListSerializerConfigSnapshot<T> extends CompositeTypeSerializerConfigSnapshot<List<T>> {
+
+	private final int VERSION = 1;
+
+	/** This empty nullary constructor is required for deserializing the configuration. */
+	public ListSerializerConfigSnapshot() {}
+
+	public ListSerializerConfigSnapshot(TypeSerializer<T> elementSerializer) {
+		super(elementSerializer);
+	}
+
+	@Override
+	public int getVersion() {
+		return VERSION;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	protected TypeSerializer<List<T>> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+		return new ListSerializer<>((TypeSerializer<T>) restoredNestedSerializers[0]);
+	}
+
+	@Override
+	protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+		return newSerializer instanceof ListSerializer;
+	}
+}
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializer.java
index 6471152b0bc..15ae10b0e73 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializer.java
@@ -19,19 +19,13 @@
 package org.apache.flink.api.common.typeutils.base;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.util.Preconditions;
 
 import java.io.IOException;
-import java.util.List;
 import java.util.Map;
 import java.util.HashMap;
 
@@ -47,7 +41,7 @@
  * @param <V> The type of the values in the map.
  */
 @Internal
-public final class MapSerializer<K, V> extends TypeSerializer<Map<K, V>> {
+public final class MapSerializer<K, V> extends CompositeTypeSerializer<Map<K, V>> {
 
 	private static final long serialVersionUID = -6885593032367050078L;
 	
@@ -64,8 +58,16 @@
 	 * @param valueSerializer The serializer for the values in the map
 	 */
 	public MapSerializer(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer) {
-		this.keySerializer = Preconditions.checkNotNull(keySerializer, "The key serializer cannot be null");
-		this.valueSerializer = Preconditions.checkNotNull(valueSerializer, "The value serializer cannot be null.");
+
+		super(
+			new MapSerializerConfigSnapshot<>(
+				Preconditions.checkNotNull(keySerializer, "The key serializer cannot be null."),
+				Preconditions.checkNotNull(valueSerializer, "The value serializer cannot be null.")),
+			keySerializer,
+			valueSerializer);
+
+		this.keySerializer = keySerializer;
+		this.valueSerializer = valueSerializer;
 	}
 
 	// ------------------------------------------------------------------------
@@ -201,44 +203,4 @@ public boolean canEqual(Object obj) {
 	public int hashCode() {
 		return keySerializer.hashCode() * 31 + valueSerializer.hashCode();
 	}
-
-	// --------------------------------------------------------------------------------------------
-	// Serializer configuration snapshotting & compatibility
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public MapSerializerConfigSnapshot snapshotConfiguration() {
-		return new MapSerializerConfigSnapshot<>(keySerializer, valueSerializer);
-	}
-
-	@Override
-	public CompatibilityResult<Map<K, V>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-		if (configSnapshot instanceof MapSerializerConfigSnapshot) {
-			List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> previousKvSerializersAndConfigs =
-				((MapSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
-
-			CompatibilityResult<K> keyCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-					previousKvSerializersAndConfigs.get(0).f0,
-					UnloadableDummyTypeSerializer.class,
-					previousKvSerializersAndConfigs.get(0).f1,
-					keySerializer);
-
-			CompatibilityResult<V> valueCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-					previousKvSerializersAndConfigs.get(1).f0,
-					UnloadableDummyTypeSerializer.class,
-					previousKvSerializersAndConfigs.get(1).f1,
-					valueSerializer);
-
-			if (!keyCompatResult.isRequiresMigration() && !valueCompatResult.isRequiresMigration()) {
-				return CompatibilityResult.compatible();
-			} else if (keyCompatResult.getConvertDeserializer() != null && valueCompatResult.getConvertDeserializer() != null) {
-				return CompatibilityResult.requiresMigration(
-					new MapSerializer<>(
-						new TypeDeserializerAdapter<>(keyCompatResult.getConvertDeserializer()),
-						new TypeDeserializerAdapter<>(valueCompatResult.getConvertDeserializer())));
-			}
-		}
-
-		return CompatibilityResult.requiresMigration();
-	}
 }
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializerConfigSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializerConfigSnapshot.java
index 9db3019ad46..5b860c21517 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializerConfigSnapshot.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializerConfigSnapshot.java
@@ -22,14 +22,16 @@
 import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 
+import java.util.Map;
+
 /**
  * Configuration snapshot for serializers of maps, containing the
  * configuration snapshot of its key serializer and value serializer.
  */
 @Internal
-public final class MapSerializerConfigSnapshot<K, V> extends CompositeTypeSerializerConfigSnapshot {
+public final class MapSerializerConfigSnapshot<K, V> extends CompositeTypeSerializerConfigSnapshot<Map<K, V>> {
 
-	private static final int VERSION = 1;
+	private static final int VERSION = 2;
 
 	/** This empty nullary constructor is required for deserializing the configuration. */
 	public MapSerializerConfigSnapshot() {}
@@ -38,8 +40,31 @@ public MapSerializerConfigSnapshot(TypeSerializer<K> keySerializer, TypeSerializ
 		super(keySerializer, valueSerializer);
 	}
 
+	@Override
+	public int[] getCompatibleVersions() {
+		return new int[]{VERSION, 1};
+	}
+
 	@Override
 	public int getVersion() {
 		return VERSION;
 	}
+
+	@Override
+	protected boolean containsSerializers() {
+		return getReadVersion() < 2;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	protected TypeSerializer<Map<K, V>> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+		return new MapSerializer<>(
+			(TypeSerializer<K>) restoredNestedSerializers[0],
+			(TypeSerializer<V>) restoredNestedSerializers[1]);
+	}
+
+	@Override
+	protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+		return newSerializer instanceof MapSerializer;
+	}
 }
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/TypeSerializerSingleton.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/TypeSerializerSingleton.java
index 9354af07e18..55e59223709 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/TypeSerializerSingleton.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/TypeSerializerSingleton.java
@@ -19,7 +19,7 @@
 package org.apache.flink.api.common.typeutils.base;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.ParameterlessTypeSerializerConfig;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
@@ -53,20 +53,20 @@ public boolean equals(Object obj) {
 	}
 
 	@Override
-	public TypeSerializerConfigSnapshot snapshotConfiguration() {
+	public TypeSerializerConfigSnapshot<T> snapshotConfiguration() {
 		// type serializer singletons should always be parameter-less
-		return new ParameterlessTypeSerializerConfig(getSerializationFormatIdentifier());
+		return new ParameterlessTypeSerializerConfig<>(getSerializationFormatIdentifier());
 	}
 
 	@Override
-	public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	public TypeSerializerSchemaCompatibility<T> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		if (configSnapshot instanceof ParameterlessTypeSerializerConfig
 				&& isCompatibleSerializationFormatIdentifier(
-						((ParameterlessTypeSerializerConfig) configSnapshot).getSerializationFormatIdentifier())) {
+						((ParameterlessTypeSerializerConfig<?>) configSnapshot).getSerializationFormatIdentifier())) {
 
-			return CompatibilityResult.compatible();
+			return TypeSerializerSchemaCompatibility.compatibleAsIs();
 		} else {
-			return CompatibilityResult.requiresMigration();
+			return TypeSerializerSchemaCompatibility.incompatible();
 		}
 	}
 
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java
index b9039690f9f..f13f4ca821e 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java
@@ -21,7 +21,7 @@
 import java.io.IOException;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.GenericTypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
@@ -142,12 +142,12 @@ public boolean canEqual(Object obj) {
 	}
 
 	@Override
-	public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	public TypeSerializerSchemaCompatibility<T> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		if (configSnapshot instanceof CopyableValueSerializerConfigSnapshot
-				&& valueClass.equals(((CopyableValueSerializerConfigSnapshot) configSnapshot).getTypeClass())) {
-			return CompatibilityResult.compatible();
+				&& valueClass.equals(((CopyableValueSerializerConfigSnapshot<?>) configSnapshot).getTypeClass())) {
+			return TypeSerializerSchemaCompatibility.compatibleAsIs();
 		} else {
-			return CompatibilityResult.requiresMigration();
+			return TypeSerializerSchemaCompatibility.incompatible();
 		}
 	}
 
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java
index 18ebcd8b63c..acd9403c63c 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java
@@ -19,19 +19,14 @@
 package org.apache.flink.api.java.typeutils.runtime;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.types.Either;
+import org.apache.flink.util.Preconditions;
 
 import java.io.IOException;
-import java.util.List;
 
 import static org.apache.flink.types.Either.Left;
 import static org.apache.flink.types.Either.Right;
@@ -43,7 +38,7 @@
  * @param <R> the Right value type
  */
 @Internal
-public class EitherSerializer<L, R> extends TypeSerializer<Either<L, R>> {
+public class EitherSerializer<L, R> extends CompositeTypeSerializer<Either<L, R>> {
 
 	private static final long serialVersionUID = 1L;
 
@@ -52,6 +47,14 @@
 	private final TypeSerializer<R> rightSerializer;
 
 	public EitherSerializer(TypeSerializer<L> leftSerializer, TypeSerializer<R> rightSerializer) {
+
+		super(
+			new EitherSerializerConfigSnapshot<>(
+				Preconditions.checkNotNull(leftSerializer),
+				Preconditions.checkNotNull(rightSerializer)),
+			leftSerializer,
+			rightSerializer);
+
 		this.leftSerializer = leftSerializer;
 		this.rightSerializer = rightSerializer;
 	}
@@ -190,46 +193,4 @@ public boolean canEqual(Object obj) {
 	public int hashCode() {
 		return 17 * leftSerializer.hashCode() + rightSerializer.hashCode();
 	}
-
-	// --------------------------------------------------------------------------------------------
-	// Serializer configuration snapshotting & compatibility
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public EitherSerializerConfigSnapshot snapshotConfiguration() {
-		return new EitherSerializerConfigSnapshot<>(leftSerializer, rightSerializer);
-	}
-
-	@Override
-	public CompatibilityResult<Either<L, R>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-		if (configSnapshot instanceof EitherSerializerConfigSnapshot) {
-			List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> previousLeftRightSerializersAndConfigs =
-				((EitherSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
-
-			CompatibilityResult<L> leftCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-					previousLeftRightSerializersAndConfigs.get(0).f0,
-					UnloadableDummyTypeSerializer.class,
-					previousLeftRightSerializersAndConfigs.get(0).f1,
-					leftSerializer);
-
-			CompatibilityResult<R> rightCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-					previousLeftRightSerializersAndConfigs.get(1).f0,
-					UnloadableDummyTypeSerializer.class,
-					previousLeftRightSerializersAndConfigs.get(1).f1,
-					rightSerializer);
-
-			if (!leftCompatResult.isRequiresMigration() && !rightCompatResult.isRequiresMigration()) {
-				return CompatibilityResult.compatible();
-			} else {
-				if (leftCompatResult.getConvertDeserializer() != null && rightCompatResult.getConvertDeserializer() != null) {
-					return CompatibilityResult.requiresMigration(
-						new EitherSerializer<>(
-							new TypeDeserializerAdapter<>(leftCompatResult.getConvertDeserializer()),
-							new TypeDeserializerAdapter<>(rightCompatResult.getConvertDeserializer())));
-				}
-			}
-		}
-
-		return CompatibilityResult.requiresMigration();
-	}
 }
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerConfigSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerConfigSnapshot.java
index f9968781a8c..bb71d684211 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerConfigSnapshot.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerConfigSnapshot.java
@@ -28,9 +28,9 @@
  * containing configuration snapshots of the Left and Right serializers.
  */
 @Internal
-public final class EitherSerializerConfigSnapshot<L, R> extends CompositeTypeSerializerConfigSnapshot {
+public final class EitherSerializerConfigSnapshot<L, R> extends CompositeTypeSerializerConfigSnapshot<Either<L, R>> {
 
-	private static final int VERSION = 1;
+	private static final int VERSION = 2;
 
 	/** This empty nullary constructor is required for deserializing the configuration. */
 	public EitherSerializerConfigSnapshot() {}
@@ -43,4 +43,27 @@ public EitherSerializerConfigSnapshot(TypeSerializer<L> leftSerializer, TypeSeri
 	public int getVersion() {
 		return VERSION;
 	}
+
+	@Override
+	public int[] getCompatibleVersions() {
+		return new int[]{VERSION, 1};
+	}
+
+	@Override
+	protected boolean containsSerializers() {
+		return getReadVersion() < 2;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	protected TypeSerializer<Either<L, R>> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+		return new EitherSerializer<>(
+			(TypeSerializer<L>) restoredNestedSerializers[0],
+			(TypeSerializer<R>) restoredNestedSerializers[1]);
+	}
+
+	@Override
+	protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+		return newSerializer instanceof EitherSerializer;
+	}
 }
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java
index a4d086d6a74..881d7f7e7da 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java
@@ -33,13 +33,13 @@
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.CompatibilityUtil;
 import org.apache.flink.api.common.typeutils.GenericTypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshotSerializationUtil;
 import org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.core.memory.ByteArrayInputStreamWithPos;
@@ -594,16 +594,14 @@ public boolean canEqual(Object obj) {
 
 	@SuppressWarnings("unchecked")
 	@Override
-	public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	public TypeSerializerSchemaCompatibility<T> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		if (configSnapshot instanceof PojoSerializerConfigSnapshot) {
 			final PojoSerializerConfigSnapshot<T> config = (PojoSerializerConfigSnapshot<T>) configSnapshot;
 
-			boolean requiresMigration = false;
-
 			if (clazz.equals(config.getTypeClass())) {
 				if (this.numFields == config.getFieldToSerializerConfigSnapshot().size()) {
 
-					CompatibilityResult<?> compatResult;
+					TypeSerializerSchemaCompatibility<?> compatResult;
 
 					// ----------- check field order and compatibility of field serializers -----------
 
@@ -622,24 +620,16 @@ public boolean canEqual(Object obj) {
 							reorderedFields[i] = fields[fieldIndex];
 
 							compatResult = CompatibilityUtil.resolveCompatibilityResult(
-								fieldToConfigSnapshotEntry.getValue().f0,
-								UnloadableDummyTypeSerializer.class,
 								fieldToConfigSnapshotEntry.getValue().f1,
 								fieldSerializers[fieldIndex]);
 
-							if (compatResult.isRequiresMigration()) {
-								requiresMigration = true;
-
-								if (compatResult.getConvertDeserializer() != null) {
-									reorderedFieldSerializers[i] = (TypeSerializer<Object>) compatResult.getConvertDeserializer();
-								} else {
-									return CompatibilityResult.requiresMigration();
-								}
+							if (compatResult.isIncompatible()) {
+								return TypeSerializerSchemaCompatibility.incompatible();
 							} else {
 								reorderedFieldSerializers[i] = fieldSerializers[fieldIndex];
 							}
 						} else {
-							return CompatibilityResult.requiresMigration();
+							return TypeSerializerSchemaCompatibility.incompatible();
 						}
 
 						i++;
@@ -671,17 +661,11 @@ public boolean canEqual(Object obj) {
 					for (Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> previousRegisteredSerializerConfig : previousRegistrations.values()) {
 						// check compatibility of subclass serializer
 						compatResult = CompatibilityUtil.resolveCompatibilityResult(
-								previousRegisteredSerializerConfig.f0,
-								UnloadableDummyTypeSerializer.class,
 								previousRegisteredSerializerConfig.f1,
 								reorderedRegisteredSubclassSerializers[i]);
 
-						if (compatResult.isRequiresMigration()) {
-							requiresMigration = true;
-
-							if (compatResult.getConvertDeserializer() == null) {
-								return CompatibilityResult.requiresMigration();
-							}
+						if (compatResult.isIncompatible()) {
+							return TypeSerializerSchemaCompatibility.incompatible();
 						}
 
 						i++;
@@ -700,19 +684,11 @@ public boolean canEqual(Object obj) {
 
 						// check compatibility of cached subclass serializer
 						compatResult = CompatibilityUtil.resolveCompatibilityResult(
-								previousCachedEntry.getValue().f0,
-								UnloadableDummyTypeSerializer.class,
 								previousCachedEntry.getValue().f1,
 								cachedSerializer);
 
-						if (compatResult.isRequiresMigration()) {
-							requiresMigration = true;
-
-							if (compatResult.getConvertDeserializer() != null) {
-								rebuiltCache.put(previousCachedEntry.getKey(), cachedSerializer);
-							} else {
-								return CompatibilityResult.requiresMigration();
-							}
+						if (compatResult.isIncompatible()) {
+							return TypeSerializerSchemaCompatibility.incompatible();
 						} else {
 							rebuiltCache.put(previousCachedEntry.getKey(), cachedSerializer);
 						}
@@ -721,31 +697,20 @@ public boolean canEqual(Object obj) {
 					// completed compatibility checks; up to this point, we can just reconfigure
 					// the serializer so that it is compatible and migration is not required
 
-					if (!requiresMigration) {
-						this.fields = reorderedFields;
-						this.fieldSerializers = reorderedFieldSerializers;
+					this.fields = reorderedFields;
+					this.fieldSerializers = reorderedFieldSerializers;
 
-						this.registeredClasses = reorderedRegisteredSubclassesToClasstags;
-						this.registeredSerializers = reorderedRegisteredSubclassSerializers;
+					this.registeredClasses = reorderedRegisteredSubclassesToClasstags;
+					this.registeredSerializers = reorderedRegisteredSubclassSerializers;
 
-						this.subclassSerializerCache = rebuiltCache;
+					this.subclassSerializerCache = rebuiltCache;
 
-						return CompatibilityResult.compatible();
-					} else {
-						return CompatibilityResult.requiresMigration(
-							new PojoSerializer<>(
-								clazz,
-								reorderedFields,
-								reorderedFieldSerializers,
-								reorderedRegisteredSubclassesToClasstags,
-								reorderedRegisteredSubclassSerializers,
-								rebuiltCache));
-					}
+					return TypeSerializerSchemaCompatibility.compatibleAsIs();
 				}
 			}
 		}
 
-		return CompatibilityResult.requiresMigration();
+		return TypeSerializerSchemaCompatibility.incompatible();
 	}
 
 	public static final class PojoSerializerConfigSnapshot<T> extends GenericTypeSerializerConfigSnapshot<T> {
@@ -838,7 +803,7 @@ public void write(DataOutputView out) throws IOException {
 					}
 
 					out.writeInt(outWithPos.getPosition());
-					TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(outViewWrapper, entry.getValue().f1);
+					TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(outViewWrapper, entry.getValue().f1);
 				}
 
 				// --- write registered subclasses and their serializers, in registration order
@@ -855,7 +820,7 @@ public void write(DataOutputView out) throws IOException {
 					}
 
 					out.writeInt(outWithPos.getPosition());
-					TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(outViewWrapper, entry.getValue().f1);
+					TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(outViewWrapper, entry.getValue().f1);
 				}
 
 				// --- write snapshot of non-registered subclass serializer cache
@@ -872,7 +837,7 @@ public void write(DataOutputView out) throws IOException {
 					}
 
 					out.writeInt(outWithPos.getPosition());
-					TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(outViewWrapper, entry.getValue().f1);
+					TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(outViewWrapper, entry.getValue().f1);
 				}
 
 				out.writeInt(outWithPos.getPosition());
@@ -927,7 +892,7 @@ public void read(DataInputView in) throws IOException {
 					fieldSerializer = TypeSerializerSerializationUtil.tryReadSerializer(inViewWrapper, getUserCodeClassLoader(), true);
 
 					inWithPos.setPosition(fieldSerializerOffsets[i * 2 + 1]);
-					fieldSerializerConfigSnapshot = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(inViewWrapper, getUserCodeClassLoader());
+					fieldSerializerConfigSnapshot = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(inViewWrapper, getUserCodeClassLoader());
 
 					fieldToSerializerConfigSnapshot.put(
 						fieldName,
@@ -953,7 +918,7 @@ public void read(DataInputView in) throws IOException {
 					registeredSubclassSerializer = TypeSerializerSerializationUtil.tryReadSerializer(inViewWrapper, getUserCodeClassLoader(), true);
 
 					inWithPos.setPosition(registeredSerializerOffsets[i * 2 + 1]);
-					registeredSubclassSerializerConfigSnapshot = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(inViewWrapper, getUserCodeClassLoader());
+					registeredSubclassSerializerConfigSnapshot = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(inViewWrapper, getUserCodeClassLoader());
 
 					this.registeredSubclassesToSerializerConfigSnapshots.put(
 						registeredSubclass,
@@ -979,7 +944,7 @@ public void read(DataInputView in) throws IOException {
 					cachedSubclassSerializer = TypeSerializerSerializationUtil.tryReadSerializer(inViewWrapper, getUserCodeClassLoader(), true);
 
 					inWithPos.setPosition(cachedSerializerOffsets[i * 2 + 1]);
-					cachedSubclassSerializerConfigSnapshot = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(inViewWrapper, getUserCodeClassLoader());
+					cachedSubclassSerializerConfigSnapshot = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(inViewWrapper, getUserCodeClassLoader());
 
 					this.nonRegisteredSubclassesToSerializerConfigSnapshots.put(
 						cachedSubclass,
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java
index 7f9cc2145be..a21d4e60a82 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java
@@ -18,14 +18,9 @@
 package org.apache.flink.api.java.typeutils.runtime;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializer;
 import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.types.Row;
@@ -33,7 +28,6 @@
 import java.io.IOException;
 import java.io.ObjectInputStream;
 import java.util.Arrays;
-import java.util.List;
 
 import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.readIntoAndCopyNullMask;
 import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.readIntoNullMask;
@@ -44,7 +38,7 @@
  * Serializer for {@link Row}.
  */
 @Internal
-public final class RowSerializer extends TypeSerializer<Row> {
+public final class RowSerializer extends CompositeTypeSerializer<Row> {
 
 	private static final long serialVersionUID = 1L;
 
@@ -56,7 +50,10 @@
 
 	@SuppressWarnings("unchecked")
 	public RowSerializer(TypeSerializer<?>[] fieldSerializers) {
-		this.fieldSerializers = (TypeSerializer<Object>[]) checkNotNull(fieldSerializers);
+
+		super(new RowSerializerConfigSnapshot(checkNotNull(fieldSerializers)), fieldSerializers);
+
+		this.fieldSerializers = (TypeSerializer<Object>[]) fieldSerializers;
 		this.arity = fieldSerializers.length;
 		this.nullMask = new boolean[fieldSerializers.length];
 	}
@@ -262,64 +259,14 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE
 	// Serializer configuration snapshotting & compatibility
 	// --------------------------------------------------------------------------------------------
 
-	@Override
-	public RowSerializerConfigSnapshot snapshotConfiguration() {
-		return new RowSerializerConfigSnapshot(fieldSerializers);
-	}
-
-	@Override
-	public CompatibilityResult<Row> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-		if (configSnapshot instanceof RowSerializerConfigSnapshot) {
-			List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> previousFieldSerializersAndConfigs =
-				((RowSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
-
-			if (previousFieldSerializersAndConfigs.size() == fieldSerializers.length) {
-				boolean requireMigration = false;
-				TypeSerializer<?>[] convertDeserializers = new TypeSerializer<?>[fieldSerializers.length];
-
-				CompatibilityResult<?> compatResult;
-				int i = 0;
-				for (Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> f : previousFieldSerializersAndConfigs) {
-					compatResult = CompatibilityUtil.resolveCompatibilityResult(
-							f.f0,
-							UnloadableDummyTypeSerializer.class,
-							f.f1,
-							fieldSerializers[i]);
-
-					if (compatResult.isRequiresMigration()) {
-						requireMigration = true;
-
-						if (compatResult.getConvertDeserializer() == null) {
-							// one of the field serializers cannot provide a fallback deserializer
-							return CompatibilityResult.requiresMigration();
-						} else {
-							convertDeserializers[i] =
-								new TypeDeserializerAdapter<>(compatResult.getConvertDeserializer());
-						}
-					}
+	public static final class RowSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot<Row> {
 
-					i++;
-				}
-
-				if (requireMigration) {
-					return CompatibilityResult.requiresMigration(new RowSerializer(convertDeserializers));
-				} else {
-					return CompatibilityResult.compatible();
-				}
-			}
-		}
-
-		return CompatibilityResult.requiresMigration();
-	}
-
-	public static final class RowSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot {
-
-		private static final int VERSION = 1;
+		private static final int VERSION = 2;
 
 		/** This empty nullary constructor is required for deserializing the configuration. */
 		public RowSerializerConfigSnapshot() {}
 
-		public RowSerializerConfigSnapshot(TypeSerializer[] fieldSerializers) {
+		public RowSerializerConfigSnapshot(TypeSerializer<?>[] fieldSerializers) {
 			super(fieldSerializers);
 		}
 
@@ -327,5 +274,25 @@ public RowSerializerConfigSnapshot(TypeSerializer[] fieldSerializers) {
 		public int getVersion() {
 			return VERSION;
 		}
+
+		@Override
+		public int[] getCompatibleVersions() {
+			return new int[]{VERSION, 1};
+		}
+
+		@Override
+		protected boolean containsSerializers() {
+			return getReadVersion() < 2;
+		}
+
+		@Override
+		protected TypeSerializer<Row> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+			return new RowSerializer(restoredNestedSerializers);
+		}
+
+		@Override
+		protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+			return newSerializer instanceof RowSerializer;
+		}
 	}
 }
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java
index 911c96f393b..d0dddb8a2d8 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializer.java
@@ -33,7 +33,10 @@
 	private static final long serialVersionUID = 1L;
 	
 	public TupleSerializer(Class<T> tupleClass, TypeSerializer<?>[] fieldSerializers) {
-		super(tupleClass, fieldSerializers);
+		super(
+			tupleClass,
+			new TupleSerializerConfigSnapshot<>(tupleClass, fieldSerializers),
+			fieldSerializers);
 	}
 
 	@Override
@@ -147,7 +150,7 @@ public T deserialize(T reuse, DataInputView source) throws IOException {
 		}
 		return reuse;
 	}
-	
+
 	private T instantiateRaw() {
 		try {
 			return tupleClass.newInstance();
@@ -156,9 +159,4 @@ private T instantiateRaw() {
 			throw new RuntimeException("Cannot instantiate tuple.", e);
 		}
 	}
-
-	@Override
-	protected TupleSerializerBase<T> createSerializerInstance(Class<T> tupleClass, TypeSerializer<?>[] fieldSerializers) {
-		return new TupleSerializer<>(tupleClass, fieldSerializers);
-	}
 }
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
index 3fb7defc2c9..bcf00631486 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerBase.java
@@ -20,25 +20,20 @@
 
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializer;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 
 import java.io.IOException;
 import java.util.Arrays;
-import java.util.List;
 import java.util.Objects;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
 
 @Internal
-public abstract class TupleSerializerBase<T> extends TypeSerializer<T> {
+public abstract class TupleSerializerBase<T> extends CompositeTypeSerializer<T> {
 
 	private static final long serialVersionUID = 1L;
 
@@ -51,9 +46,15 @@
 	private int length = -2;
 
 	@SuppressWarnings("unchecked")
-	public TupleSerializerBase(Class<T> tupleClass, TypeSerializer<?>[] fieldSerializers) {
+	public TupleSerializerBase(
+			Class<T> tupleClass,
+			CompositeTypeSerializerConfigSnapshot<T> serializerConfigSnapshot,
+			TypeSerializer<?>[] fieldSerializers) {
+
+		super(serializerConfigSnapshot, fieldSerializers);
+
 		this.tupleClass = checkNotNull(tupleClass);
-		this.fieldSerializers = (TypeSerializer<Object>[]) checkNotNull(fieldSerializers);
+		this.fieldSerializers = (TypeSerializer<Object>[]) fieldSerializers;
 		this.arity = fieldSerializers.length;
 	}
 	
@@ -124,67 +125,6 @@ public boolean canEqual(Object obj) {
 		return obj instanceof TupleSerializerBase;
 	}
 
-	// --------------------------------------------------------------------------------------------
-	// Serializer configuration snapshotting & compatibility
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public TupleSerializerConfigSnapshot<T> snapshotConfiguration() {
-		return new TupleSerializerConfigSnapshot<>(tupleClass, fieldSerializers);
-	}
-
-	@SuppressWarnings("unchecked")
-	@Override
-	public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-		if (configSnapshot instanceof TupleSerializerConfigSnapshot) {
-			final TupleSerializerConfigSnapshot<T> config = (TupleSerializerConfigSnapshot<T>) configSnapshot;
-
-			if (tupleClass.equals(config.getTupleClass())) {
-				List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> previousFieldSerializersAndConfigs =
-					((TupleSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
-
-				if (previousFieldSerializersAndConfigs.size() == fieldSerializers.length) {
-
-					TypeSerializer<Object>[] convertFieldSerializers = new TypeSerializer[fieldSerializers.length];
-					boolean requiresMigration = false;
-					CompatibilityResult<Object> compatResult;
-					int i = 0;
-					for (Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> f : previousFieldSerializersAndConfigs) {
-						compatResult = CompatibilityUtil.resolveCompatibilityResult(
-								f.f0,
-								UnloadableDummyTypeSerializer.class,
-								f.f1,
-								fieldSerializers[i]);
-
-						if (compatResult.isRequiresMigration()) {
-							requiresMigration = true;
-
-							if (compatResult.getConvertDeserializer() != null) {
-								convertFieldSerializers[i] =
-									new TypeDeserializerAdapter<>(compatResult.getConvertDeserializer());
-							} else {
-								return CompatibilityResult.requiresMigration();
-							}
-						}
-
-						i++;
-					}
-
-					if (!requiresMigration) {
-						return CompatibilityResult.compatible();
-					} else {
-						return CompatibilityResult.requiresMigration(
-							createSerializerInstance(tupleClass, convertFieldSerializers));
-					}
-				}
-			}
-		}
-
-		return CompatibilityResult.requiresMigration();
-	}
-
-	protected abstract TupleSerializerBase<T> createSerializerInstance(Class<T> tupleClass, TypeSerializer<?>[] fieldSerializers);
-
 	@VisibleForTesting
 	public TypeSerializer<Object>[] getFieldSerializers() {
 		return fieldSerializers;
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerConfigSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerConfigSnapshot.java
index eac5200da9c..48c484de8bd 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerConfigSnapshot.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerConfigSnapshot.java
@@ -21,6 +21,8 @@
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
+import org.apache.flink.api.java.tuple.Tuple;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.util.InstantiationUtil;
@@ -29,12 +31,12 @@
 import java.io.IOException;
 
 /**
- * Snapshot of a tuple serializer's configuration.
+ * Snapshot of a {@link TupleSerializer}'s configuration.
  */
 @Internal
-public final class TupleSerializerConfigSnapshot<T> extends CompositeTypeSerializerConfigSnapshot {
+public final class TupleSerializerConfigSnapshot<T extends Tuple> extends CompositeTypeSerializerConfigSnapshot<T> {
 
-	private static final int VERSION = 1;
+	private static final int VERSION = 2;
 
 	private Class<T> tupleClass;
 
@@ -72,6 +74,39 @@ public int getVersion() {
 		return VERSION;
 	}
 
+	@Override
+	public int[] getCompatibleVersions() {
+		return new int[]{VERSION, 1};
+	}
+
+	@Override
+	protected boolean containsSerializers() {
+		return getReadVersion() < 2;
+	}
+
+	@Override
+	protected TypeSerializer<T> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+		return new TupleSerializer<>(tupleClass, restoredNestedSerializers);
+	}
+
+	@Override
+	protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+		return newSerializer instanceof TupleSerializer;
+	}
+
+	@Override
+	public TypeSerializerSchemaCompatibility<T> resolveSchemaCompatibility(TypeSerializer<?> newSerializer) {
+		TypeSerializerSchemaCompatibility<T> result = super.resolveSchemaCompatibility(newSerializer);
+
+		final TupleSerializer<T> tupleSerializer = (TupleSerializer<T>) newSerializer;
+
+		if (result.isIncompatible() && !tupleClass.equals(tupleSerializer.getTupleClass())) {
+			return TypeSerializerSchemaCompatibility.incompatible();
+		}
+
+		return result;
+	}
+
 	public Class<T> getTupleClass() {
 		return tupleClass;
 	}
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java
index 0a028ebd79d..7f99ff55b7f 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java
@@ -23,7 +23,7 @@
 import java.util.LinkedHashMap;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.core.memory.DataInputView;
@@ -182,18 +182,18 @@ public boolean canEqual(Object obj) {
 
 	@SuppressWarnings("unchecked")
 	@Override
-	public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	public TypeSerializerSchemaCompatibility<T> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		if (configSnapshot instanceof ValueSerializerConfigSnapshot) {
 			final ValueSerializerConfigSnapshot<T> config = (ValueSerializerConfigSnapshot<T>) configSnapshot;
 
 			if (type.equals(config.getTypeClass())) {
 				// currently, simply checking the type of the value class is sufficient;
 				// in the future, if there are more Kryo registrations, we should try to resolve that
-				return CompatibilityResult.compatible();
+				return TypeSerializerSchemaCompatibility.compatibleAsIs();
 			}
 		}
 
-		return CompatibilityResult.requiresMigration();
+		return TypeSerializerSchemaCompatibility.incompatible();
 	}
 
 	public static class ValueSerializerConfigSnapshot<T extends Value> extends KryoRegistrationSerializerConfigSnapshot<T> {
diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
index d8158aabb92..adfa9868936 100644
--- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
+++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializer.java
@@ -20,7 +20,7 @@
 
 import org.apache.flink.annotation.VisibleForTesting;
 import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.api.java.typeutils.AvroUtils;
@@ -463,7 +463,7 @@ private void checkKryoInitialized() {
 
 	@SuppressWarnings("unchecked")
 	@Override
-	public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	public TypeSerializerSchemaCompatibility<T> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		if (configSnapshot instanceof KryoSerializerConfigSnapshot) {
 			final KryoSerializerConfigSnapshot<T> config = (KryoSerializerConfigSnapshot<T>) configSnapshot;
 
@@ -483,18 +483,18 @@ private void checkKryoInitialized() {
 							"proper serializer, because its previous serializer cannot be loaded or is no " +
 							"longer valid but a new serializer is not available", reconfiguredRegistrationEntry.getKey());
 
-						return CompatibilityResult.requiresMigration();
+						return TypeSerializerSchemaCompatibility.incompatible();
 					}
 				}
 
 				// there's actually no way to tell if new Kryo serializers are compatible with
 				// the previous ones they overwrite; we can only signal compatibility and hope for the best
 				this.kryoRegistrations = reconfiguredRegistrations;
-				return CompatibilityResult.compatible();
+				return TypeSerializerSchemaCompatibility.compatibleAsIs();
 			}
 		}
 
-		return CompatibilityResult.requiresMigration();
+		return TypeSerializerSchemaCompatibility.incompatible();
 	}
 
 	public static final class KryoSerializerConfigSnapshot<T> extends KryoRegistrationSerializerConfigSnapshot<T> {
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java
index 57015c78be0..8ee7193ddfd 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/SerializerTestBase.java
@@ -105,27 +105,27 @@ public void testConfigSnapshotInstantiation() {
 
 	@Test
 	public void testSnapshotConfigurationAndReconfigure() throws Exception {
-		final TypeSerializerConfigSnapshot configSnapshot = getSerializer().snapshotConfiguration();
+		final TypeSerializerConfigSnapshot<T> configSnapshot = getSerializer().snapshotConfiguration();
 
 		byte[] serializedConfig;
 		try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
-			TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(
 				new DataOutputViewStreamWrapper(out), configSnapshot);
 			serializedConfig = out.toByteArray();
 		}
 
-		TypeSerializerConfigSnapshot restoredConfig;
+		TypeSerializerConfigSnapshot<T> restoredConfig;
 		try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) {
-			restoredConfig = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
+			restoredConfig = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(
 				new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader());
 		}
 
-		CompatibilityResult strategy = getSerializer().ensureCompatibility(restoredConfig);
-		assertFalse(strategy.isRequiresMigration());
+		TypeSerializerSchemaCompatibility strategy = restoredConfig.resolveSchemaCompatibility(getSerializer());
+		assertFalse(strategy.isIncompatible());
 
 		// also verify that the serializer's reconfigure implementation detects incompatibility
-		strategy = getSerializer().ensureCompatibility(new TestIncompatibleSerializerConfigSnapshot());
-		assertTrue(strategy.isRequiresMigration());
+		strategy = new TestIncompatibleSerializerConfigSnapshot<>().resolveSchemaCompatibility(getSerializer());
+		assertTrue(strategy.isIncompatible());
 	}
 	
 	@Test
@@ -526,7 +526,7 @@ public void skipBytesToRead(int numBytes) throws IOException {
 		}
 	}
 
-	public static final class TestIncompatibleSerializerConfigSnapshot extends TypeSerializerConfigSnapshot {
+	public static final class TestIncompatibleSerializerConfigSnapshot<T> extends TypeSerializerConfigSnapshot<T> {
 		@Override
 		public int getVersion() {
 			return 0;
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtilTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtilTest.java
index 04739314dcc..a1d45b0a1e4 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtilTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtilTest.java
@@ -148,31 +148,30 @@ public void testSerializerSerializationWithInvalidClass() throws Exception {
 	 */
 	@Test
 	public void testSerializeConfigurationSnapshots() throws Exception {
-		TypeSerializerSerializationUtilTest.TestConfigSnapshot configSnapshot1 =
-			new TypeSerializerSerializationUtilTest.TestConfigSnapshot(1, "foo");
+		TypeSerializerSerializationUtilTest.TestConfigSnapshot<String> configSnapshot1 =
+			new TypeSerializerSerializationUtilTest.TestConfigSnapshot<>(1, "foo");
 
-		TypeSerializerSerializationUtilTest.TestConfigSnapshot configSnapshot2 =
-			new TypeSerializerSerializationUtilTest.TestConfigSnapshot(2, "bar");
+		TypeSerializerSerializationUtilTest.TestConfigSnapshot<String> configSnapshot2 =
+			new TypeSerializerSerializationUtilTest.TestConfigSnapshot<>(2, "bar");
 
 		byte[] serializedConfig;
 		try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
-			TypeSerializerSerializationUtil.writeSerializerConfigSnapshots(
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshots(
 				new DataOutputViewStreamWrapper(out),
-				configSnapshot1,
-				configSnapshot2);
+				Arrays.asList(configSnapshot1, configSnapshot2));
 
 			serializedConfig = out.toByteArray();
 		}
 
-		TypeSerializerConfigSnapshot[] restoredConfigs;
+		List<TypeSerializerConfigSnapshot<?>> restoredConfigs;
 		try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) {
-			restoredConfigs = TypeSerializerSerializationUtil.readSerializerConfigSnapshots(
+			restoredConfigs = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshots(
 				new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader());
 		}
 
-		assertEquals(2, restoredConfigs.length);
-		assertEquals(configSnapshot1, restoredConfigs[0]);
-		assertEquals(configSnapshot2, restoredConfigs[1]);
+		assertEquals(2, restoredConfigs.size());
+		assertEquals(configSnapshot1, restoredConfigs.get(0));
+		assertEquals(configSnapshot2, restoredConfigs.get(1));
 	}
 
 	/**
@@ -182,14 +181,14 @@ public void testSerializeConfigurationSnapshots() throws Exception {
 	public void testFailsWhenConfigurationSnapshotClassNotFound() throws Exception {
 		byte[] serializedConfig;
 		try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
-			TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(
-				new DataOutputViewStreamWrapper(out), new TypeSerializerSerializationUtilTest.TestConfigSnapshot(123, "foobar"));
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(
+				new DataOutputViewStreamWrapper(out), new TypeSerializerSerializationUtilTest.TestConfigSnapshot<>(123, "foobar"));
 			serializedConfig = out.toByteArray();
 		}
 
 		try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) {
 			// read using a dummy classloader
-			TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
+			TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(
 				new DataInputViewStreamWrapper(in), new URLClassLoader(new URL[0], null));
 			fail("Expected a ClassNotFoundException wrapped in IOException");
 		} catch (IOException expected) {
@@ -266,7 +265,7 @@ public void testAnonymousSerializerClassWithChangedSerialVersionUID() throws Exc
 		Assert.assertTrue(anonymousClassSerializer.getClass().isAnonymousClass());
 	}
 
-	public static class TestConfigSnapshot extends TypeSerializerConfigSnapshot {
+	public static class TestConfigSnapshot<T> extends TypeSerializerConfigSnapshot<T> {
 
 		static final int VERSION = 1;
 
@@ -411,13 +410,14 @@ public void serialize(Integer record, DataOutputView target) throws IOException
 		}
 
 		@Override
-		public TypeSerializerConfigSnapshot snapshotConfiguration() {
+		public TypeSerializerConfigSnapshot<Integer> snapshotConfiguration() {
 			return IntSerializer.INSTANCE.snapshotConfiguration();
 		}
 
 		@Override
-		public CompatibilityResult<Integer> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-			return IntSerializer.INSTANCE.ensureCompatibility(configSnapshot);
+		public TypeSerializerSchemaCompatibility<Integer> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
+			// TODO this method will be removed in follow-up commits
+			return null;
 		}
 
 		@Override
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerTest.java
index e3ce3ee6fca..fadac091dde 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerTest.java
@@ -18,10 +18,10 @@
 
 package org.apache.flink.api.common.typeutils.base;
 
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.SerializerTestInstance;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshotSerializationUtil;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.util.InstantiationUtil;
@@ -72,9 +72,10 @@ public void testReconfiguration() {
 		assertEquals(PublicEnum.PAULA.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.PAULA).intValue());
 
 		// reconfigure and verify compatibility
-		CompatibilityResult<PublicEnum> compatResult = serializer.ensureCompatibility(
-			new EnumSerializer.EnumSerializerConfigSnapshot<>(PublicEnum.class, mockPreviousOrder));
-		assertFalse(compatResult.isRequiresMigration());
+		TypeSerializerSchemaCompatibility<PublicEnum> compatResult =
+			new EnumSerializer.EnumSerializerConfigSnapshot<>(PublicEnum.class, mockPreviousOrder)
+				.resolveSchemaCompatibility(serializer);
+		assertFalse(compatResult.isIncompatible());
 
 		// after reconfiguration, the order should be first the original BAR, PAULA, NATHANIEL,
 		// followed by the "new enum constants" FOO, PETER, EMMA
@@ -95,19 +96,19 @@ public void testConfigurationSnapshotSerialization() throws Exception {
 
 		byte[] serializedConfig;
 		try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
-			TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(
 				new DataOutputViewStreamWrapper(out), serializer.snapshotConfiguration());
 			serializedConfig = out.toByteArray();
 		}
 
-		TypeSerializerConfigSnapshot restoredConfig;
+		TypeSerializerConfigSnapshot<PublicEnum> restoredConfig;
 		try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) {
-			restoredConfig = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
+			restoredConfig = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(
 				new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader());
 		}
 
-		CompatibilityResult<PublicEnum> compatResult = serializer.ensureCompatibility(restoredConfig);
-		assertFalse(compatResult.isRequiresMigration());
+		TypeSerializerSchemaCompatibility<PublicEnum> compatResult = restoredConfig.resolveSchemaCompatibility(serializer);
+		assertFalse(compatResult.isIncompatible());
 
 		assertEquals(PublicEnum.FOO.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.FOO).intValue());
 		assertEquals(PublicEnum.BAR.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.BAR).intValue());
@@ -161,9 +162,10 @@ public void testSerializeReconfiguredEnumSerializer() throws Exception {
 		assertEquals(PublicEnum.PAULA.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.PAULA).intValue());
 
 		// reconfigure and verify compatibility
-		CompatibilityResult<PublicEnum> compatResult = serializer.ensureCompatibility(
-			new EnumSerializer.EnumSerializerConfigSnapshot<>(PublicEnum.class, mockPreviousOrder));
-		assertFalse(compatResult.isRequiresMigration());
+		TypeSerializerSchemaCompatibility<PublicEnum> compatResult =
+			new EnumSerializer.EnumSerializerConfigSnapshot<>(PublicEnum.class, mockPreviousOrder)
+				.resolveSchemaCompatibility(serializer);
+		assertFalse(compatResult.isIncompatible());
 
 		// serialize and deserialize again the serializer
 		byte[] serializedSerializer = InstantiationUtil.serializeObject(serializer);
diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerUpgradeTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerUpgradeTest.java
index 1f67acbab7d..fb052b4ef93 100644
--- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerUpgradeTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerUpgradeTest.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.api.common.typeutils.base;
 
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshotSerializationUtil;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.util.TestLogger;
@@ -56,7 +56,7 @@
 	 */
 	@Test
 	public void checkIndenticalEnums() throws Exception {
-		Assert.assertFalse(checkCompatibility(ENUM_A, ENUM_A).isRequiresMigration());
+		Assert.assertFalse(checkCompatibility(ENUM_A, ENUM_A).isIncompatible());
 	}
 
 	/**
@@ -64,7 +64,7 @@ public void checkIndenticalEnums() throws Exception {
 	 */
 	@Test
 	public void checkAppendedField() throws Exception {
-		Assert.assertFalse(checkCompatibility(ENUM_A, ENUM_B).isRequiresMigration());
+		Assert.assertFalse(checkCompatibility(ENUM_A, ENUM_B).isIncompatible());
 	}
 
 	/**
@@ -72,7 +72,7 @@ public void checkAppendedField() throws Exception {
 	 */
 	@Test
 	public void checkRemovedField() throws Exception {
-		Assert.assertTrue(checkCompatibility(ENUM_A, ENUM_C).isRequiresMigration());
+		Assert.assertTrue(checkCompatibility(ENUM_A, ENUM_C).isIncompatible());
 	}
 
 	/**
@@ -80,11 +80,11 @@ public void checkRemovedField() throws Exception {
 	 */
 	@Test
 	public void checkDifferentFieldOrder() throws Exception {
-		Assert.assertFalse(checkCompatibility(ENUM_A, ENUM_D).isRequiresMigration());
+		Assert.assertFalse(checkCompatibility(ENUM_A, ENUM_D).isIncompatible());
 	}
 
 	@SuppressWarnings("unchecked")
-	private static CompatibilityResult checkCompatibility(String enumSourceA, String enumSourceB)
+	private static TypeSerializerSchemaCompatibility checkCompatibility(String enumSourceA, String enumSourceB)
 		throws IOException, ClassNotFoundException {
 
 		ClassLoader classLoader = compileAndLoadEnum(
@@ -98,7 +98,7 @@ private static CompatibilityResult checkCompatibility(String enumSourceA, String
 			ByteArrayOutputStream outBuffer = new ByteArrayOutputStream();
 			DataOutputViewStreamWrapper outputViewStreamWrapper = new DataOutputViewStreamWrapper(outBuffer)) {
 
-			TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(outputViewStreamWrapper, snapshot);
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(outputViewStreamWrapper, snapshot);
 			snapshotBytes = outBuffer.toByteArray();
 		}
 
@@ -110,11 +110,11 @@ private static CompatibilityResult checkCompatibility(String enumSourceA, String
 			ByteArrayInputStream inBuffer = new ByteArrayInputStream(snapshotBytes);
 			DataInputViewStreamWrapper inputViewStreamWrapper = new DataInputViewStreamWrapper(inBuffer)) {
 
-			restoredSnapshot = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(inputViewStreamWrapper, classLoader2);
+			restoredSnapshot = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(inputViewStreamWrapper, classLoader2);
 		}
 
 		EnumSerializer enumSerializer2 = new EnumSerializer(classLoader2.loadClass(ENUM_NAME));
-		return enumSerializer2.ensureCompatibility(restoredSnapshot);
+		return restoredSnapshot.resolveSchemaCompatibility(enumSerializer2);
 	}
 
 	private static ClassLoader compileAndLoadEnum(File root, String filename, String source) throws IOException {
diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializerTest.java
index bdb3f8fb3e4..c36b58abfc3 100644
--- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializerTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializerTest.java
@@ -34,7 +34,7 @@
 
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.SerializerTestBase;
 import org.apache.flink.api.common.typeutils.TypeComparator;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
@@ -42,7 +42,7 @@
 import org.apache.flink.api.common.operators.Keys.ExpressionKeys;
 import org.apache.flink.api.common.operators.Keys.IncompatibleKeysException;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshotSerializationUtil;
 import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.api.common.typeutils.base.DateSerializer;
 import org.apache.flink.api.common.typeutils.base.DoubleSerializer;
@@ -294,10 +294,10 @@ public void testReconfigureWithDifferentPojoType() throws Exception {
 			TypeExtractor.getForClass(SubTestUserClassB.class).createSerializer(new ExecutionConfig());
 
 		// snapshot configuration and serialize to bytes
-		TypeSerializerConfigSnapshot pojoSerializerConfigSnapshot = pojoSerializer1.snapshotConfiguration();
+		TypeSerializerConfigSnapshot<SubTestUserClassB> pojoSerializerConfigSnapshot = pojoSerializer1.snapshotConfiguration();
 		byte[] serializedConfig;
 		try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
-			TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), pojoSerializerConfigSnapshot);
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), pojoSerializerConfigSnapshot);
 			serializedConfig = out.toByteArray();
 		}
 
@@ -306,12 +306,13 @@ public void testReconfigureWithDifferentPojoType() throws Exception {
 
 		// read configuration again from bytes
 		try(ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) {
-			pojoSerializerConfigSnapshot = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
+			pojoSerializerConfigSnapshot = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(
 				new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader());
 		}
 
-		CompatibilityResult<SubTestUserClassA> compatResult = pojoSerializer2.ensureCompatibility(pojoSerializerConfigSnapshot);
-		assertTrue(compatResult.isRequiresMigration());
+		TypeSerializerSchemaCompatibility<SubTestUserClassB> compatResult = pojoSerializerConfigSnapshot
+			.resolveSchemaCompatibility(pojoSerializer2);
+		assertTrue(compatResult.isIncompatible());
 	}
 
 	/**
@@ -330,10 +331,10 @@ public void testReconfigureDifferentSubclassRegistrationOrder() throws Exception
 		int subClassBTag = pojoSerializer.getRegisteredClasses().get(SubTestUserClassB.class);
 
 		// snapshot configuration and serialize to bytes
-		TypeSerializerConfigSnapshot pojoSerializerConfigSnapshot = pojoSerializer.snapshotConfiguration();
+		TypeSerializerConfigSnapshot<TestUserClass> pojoSerializerConfigSnapshot = pojoSerializer.snapshotConfiguration();
 		byte[] serializedConfig;
 		try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
-			TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), pojoSerializerConfigSnapshot);
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), pojoSerializerConfigSnapshot);
 			serializedConfig = out.toByteArray();
 		}
 
@@ -346,12 +347,13 @@ public void testReconfigureDifferentSubclassRegistrationOrder() throws Exception
 
 		// read configuration from bytes
 		try(ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) {
-			pojoSerializerConfigSnapshot = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
+			pojoSerializerConfigSnapshot = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(
 				new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader());
 		}
 
-		CompatibilityResult<TestUserClass> compatResult = pojoSerializer.ensureCompatibility(pojoSerializerConfigSnapshot);
-		assertTrue(!compatResult.isRequiresMigration());
+		TypeSerializerSchemaCompatibility<TestUserClass> compatResult = pojoSerializerConfigSnapshot
+			.resolveSchemaCompatibility(pojoSerializer);
+		assertTrue(!compatResult.isIncompatible());
 
 		// reconfigure - check reconfiguration result and that registration ids remains the same
 		//assertEquals(ReconfigureResult.COMPATIBLE, pojoSerializer.reconfigure(pojoSerializerConfigSnapshot));
@@ -376,10 +378,10 @@ public void testReconfigureRepopulateNonregisteredSubclassSerializerCache() thro
 		assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassB.class));
 
 		// snapshot configuration and serialize to bytes
-		TypeSerializerConfigSnapshot pojoSerializerConfigSnapshot = pojoSerializer.snapshotConfiguration();
+		TypeSerializerConfigSnapshot<TestUserClass> pojoSerializerConfigSnapshot = pojoSerializer.snapshotConfiguration();
 		byte[] serializedConfig;
 		try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
-			TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), pojoSerializerConfigSnapshot);
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), pojoSerializerConfigSnapshot);
 			serializedConfig = out.toByteArray();
 		}
 
@@ -389,13 +391,14 @@ public void testReconfigureRepopulateNonregisteredSubclassSerializerCache() thro
 
 		// read configuration from bytes
 		try(ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) {
-			pojoSerializerConfigSnapshot = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
+			pojoSerializerConfigSnapshot = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(
 				new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader());
 		}
 
 		// reconfigure - check reconfiguration result and that subclass serializer cache is repopulated
-		CompatibilityResult<TestUserClass> compatResult = pojoSerializer.ensureCompatibility(pojoSerializerConfigSnapshot);
-		assertFalse(compatResult.isRequiresMigration());
+		TypeSerializerSchemaCompatibility<TestUserClass> compatResult = pojoSerializerConfigSnapshot
+			.resolveSchemaCompatibility(pojoSerializer);
+		assertFalse(compatResult.isIncompatible());
 		assertEquals(2, pojoSerializer.getSubclassSerializerCache().size());
 		assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassA.class));
 		assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassB.class));
@@ -434,10 +437,10 @@ public void testReconfigureWithPreviouslyNonregisteredSubclasses() throws Except
 		assertEquals(0, pojoSerializer.getRegisteredSerializers().length);
 
 		// snapshot configuration and serialize to bytes
-		TypeSerializerConfigSnapshot pojoSerializerConfigSnapshot = pojoSerializer.snapshotConfiguration();
+		TypeSerializerConfigSnapshot<TestUserClass> pojoSerializerConfigSnapshot = pojoSerializer.snapshotConfiguration();
 		byte[] serializedConfig;
 		try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
-			TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), pojoSerializerConfigSnapshot);
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), pojoSerializerConfigSnapshot);
 			serializedConfig = out.toByteArray();
 		}
 
@@ -449,15 +452,16 @@ public void testReconfigureWithPreviouslyNonregisteredSubclasses() throws Except
 
 		// read configuration from bytes
 		try(ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) {
-			pojoSerializerConfigSnapshot = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
+			pojoSerializerConfigSnapshot = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(
 				new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader());
 		}
 
 		// reconfigure - check reconfiguration result and that
 		// 1) subclass serializer cache is repopulated
 		// 2) registrations also contain the now registered subclasses
-		CompatibilityResult<TestUserClass> compatResult = pojoSerializer.ensureCompatibility(pojoSerializerConfigSnapshot);
-		assertFalse(compatResult.isRequiresMigration());
+		TypeSerializerSchemaCompatibility<TestUserClass> compatResult = pojoSerializerConfigSnapshot
+			.resolveSchemaCompatibility(pojoSerializer);
+		assertFalse(compatResult.isIncompatible());
 		assertEquals(2, pojoSerializer.getSubclassSerializerCache().size());
 		assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassA.class));
 		assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassB.class));
@@ -533,8 +537,9 @@ public void testReconfigureWithDifferentFieldOrder() throws Exception {
 				new HashMap<Class<?>, Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>>()); // empty; irrelevant for this test
 
 		// reconfigure - check reconfiguration result and that fields are reordered to the previous order
-		CompatibilityResult<TestUserClass> compatResult = pojoSerializer.ensureCompatibility(mockPreviousConfigSnapshot);
-		assertFalse(compatResult.isRequiresMigration());
+		TypeSerializerSchemaCompatibility<TestUserClass> compatResult = mockPreviousConfigSnapshot
+			.resolveSchemaCompatibility(pojoSerializer);
+		assertFalse(compatResult.isIncompatible());
 		int i = 0;
 		for (Field field : mockOriginalFieldOrder) {
 			assertEquals(field, pojoSerializer.getFields()[i]);
@@ -552,7 +557,7 @@ public void testSerializerSerializationFailureResilience() throws Exception{
 		byte[] serializedConfig;
 		try (
 			ByteArrayOutputStream out = new ByteArrayOutputStream()) {
-			TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), config);
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), config);
 			serializedConfig = out.toByteArray();
 		}
 
@@ -567,14 +572,14 @@ public void testSerializerSerializationFailureResilience() throws Exception{
 		PojoSerializer.PojoSerializerConfigSnapshot<TestUserClass> deserializedConfig;
 		try(ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) {
 			deserializedConfig = (PojoSerializer.PojoSerializerConfigSnapshot<TestUserClass>)
-				TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
+				TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(
 					new DataInputViewStreamWrapper(in),
 					new ArtificialCNFExceptionThrowingClassLoader(
 						Thread.currentThread().getContextClassLoader(),
 						cnfThrowingClassnames));
 		}
 
-		Assert.assertFalse(pojoSerializer.ensureCompatibility(deserializedConfig).isRequiresMigration());
+		Assert.assertFalse(deserializedConfig.resolveSchemaCompatibility(pojoSerializer).isIncompatible());
 		verifyPojoSerializerConfigSnapshotWithSerializerSerializationFailure(config, deserializedConfig);
 	}
 
diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java
index 89e9ec3f8c9..a226b6a1aad 100644
--- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java
+++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerCompatibilityTest.java
@@ -19,9 +19,10 @@
 package org.apache.flink.api.java.typeutils.runtime.kryo;
 
 import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshotSerializationUtil;
 import org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
@@ -61,13 +62,13 @@ public void testMigrationStrategyForRemovedAvroDependency() throws Exception {
 		KryoSerializer<TestClass> kryoSerializerForA = new KryoSerializer<>(TestClass.class, new ExecutionConfig());
 
 		// read configuration again from bytes
-		TypeSerializerConfigSnapshot kryoSerializerConfigSnapshot;
+		TypeSerializerConfigSnapshot<TestClass> kryoSerializerConfigSnapshot;
 		try (InputStream in = getClass().getResourceAsStream("/kryo-serializer-flink1.3-snapshot")) {
-			kryoSerializerConfigSnapshot = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
+			kryoSerializerConfigSnapshot = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(
 				new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader());
 		}
-		CompatibilityResult<TestClass> compatResult = kryoSerializerForA.ensureCompatibility(kryoSerializerConfigSnapshot);
-		assertFalse(compatResult.isRequiresMigration());
+		TypeSerializerSchemaCompatibility<TestClass> compatResult = kryoSerializerConfigSnapshot.resolveSchemaCompatibility(kryoSerializerForA);
+		assertFalse(compatResult.isIncompatible());
 	}
 
 	@Test
@@ -94,10 +95,10 @@ public void testMigrationStrategyWithDifferentKryoType() throws Exception {
 		KryoSerializer<TestClassA> kryoSerializerForA = new KryoSerializer<>(TestClassA.class, new ExecutionConfig());
 
 		// snapshot configuration and serialize to bytes
-		TypeSerializerConfigSnapshot kryoSerializerConfigSnapshot = kryoSerializerForA.snapshotConfiguration();
+		TypeSerializerConfigSnapshot<TestClassA> kryoSerializerConfigSnapshot = kryoSerializerForA.snapshotConfiguration();
 		byte[] serializedConfig;
 		try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
-			TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), kryoSerializerConfigSnapshot);
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), kryoSerializerConfigSnapshot);
 			serializedConfig = out.toByteArray();
 		}
 
@@ -105,12 +106,12 @@ public void testMigrationStrategyWithDifferentKryoType() throws Exception {
 
 		// read configuration again from bytes
 		try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) {
-			kryoSerializerConfigSnapshot = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
+			kryoSerializerConfigSnapshot = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(
 				new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader());
 		}
 
-		CompatibilityResult<TestClassB> compatResult = kryoSerializerForB.ensureCompatibility(kryoSerializerConfigSnapshot);
-		assertTrue(compatResult.isRequiresMigration());
+		TypeSerializerSchemaCompatibility<TestClassA> compatResult = kryoSerializerConfigSnapshot.resolveSchemaCompatibility(kryoSerializerForB);
+		assertTrue(compatResult.isIncompatible());
 	}
 
 	@Test
@@ -248,10 +249,10 @@ public void testMigrationStrategyForDifferentRegistrationOrder() throws Exceptio
 		int testClassBId = kryoSerializer.getKryo().getRegistration(TestClassB.class).getId();
 
 		// snapshot configuration and serialize to bytes
-		TypeSerializerConfigSnapshot kryoSerializerConfigSnapshot = kryoSerializer.snapshotConfiguration();
+		TypeSerializerConfigSnapshot<TestClass> kryoSerializerConfigSnapshot = kryoSerializer.snapshotConfiguration();
 		byte[] serializedConfig;
 		try (ByteArrayOutputStream out = new ByteArrayOutputStream()) {
-			TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), kryoSerializerConfigSnapshot);
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), kryoSerializerConfigSnapshot);
 			serializedConfig = out.toByteArray();
 		}
 
@@ -264,13 +265,13 @@ public void testMigrationStrategyForDifferentRegistrationOrder() throws Exceptio
 
 		// read configuration from bytes
 		try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) {
-			kryoSerializerConfigSnapshot = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
+			kryoSerializerConfigSnapshot = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(
 				new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader());
 		}
 
 		// reconfigure - check reconfiguration result and that registration id remains the same
-		CompatibilityResult<TestClass> compatResult = kryoSerializer.ensureCompatibility(kryoSerializerConfigSnapshot);
-		assertFalse(compatResult.isRequiresMigration());
+		TypeSerializerSchemaCompatibility<TestClass> compatResult = kryoSerializerConfigSnapshot.resolveSchemaCompatibility(kryoSerializer);
+		assertFalse(compatResult.isIncompatible());
 		assertEquals(testClassId, kryoSerializer.getKryo().getRegistration(TestClass.class).getId());
 		assertEquals(testClassAId, kryoSerializer.getKryo().getRegistration(TestClassA.class).getId());
 		assertEquals(testClassBId, kryoSerializer.getKryo().getRegistration(TestClassB.class).getId());
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java
index f3801991be6..ef4c1c16f4e 100644
--- a/flink-formats/flink-avro/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java
@@ -20,9 +20,9 @@
 
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -225,7 +225,7 @@ public boolean canEqual(Object obj) {
 
 	@SuppressWarnings("unchecked")
 	@Override
-	public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	public TypeSerializerSchemaCompatibility<T> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		if (configSnapshot instanceof AvroSerializerConfigSnapshot) {
 			final AvroSerializerConfigSnapshot<T> config = (AvroSerializerConfigSnapshot<T>) configSnapshot;
 
@@ -238,18 +238,18 @@ public boolean canEqual(Object obj) {
 
 				for (Map.Entry<String, KryoRegistration> reconfiguredRegistrationEntry : kryoRegistrations.entrySet()) {
 					if (reconfiguredRegistrationEntry.getValue().isDummy()) {
-						return CompatibilityResult.requiresMigration();
+						return TypeSerializerSchemaCompatibility.incompatible();
 					}
 				}
 
 				this.kryoRegistrations = oldRegistrations;
-				return CompatibilityResult.compatible();
+				return TypeSerializerSchemaCompatibility.compatibleAsIs();
 			}
 		}
 
 		// ends up here if the preceding serializer is not
 		// the ValueSerializer, or serialized data type has changed
-		return CompatibilityResult.requiresMigration();
+		return TypeSerializerSchemaCompatibility.incompatible();
 	}
 
 	/**
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.java
index b313625bfe2..5b152fb56ce 100644
--- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.java
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroSerializer.java
@@ -18,9 +18,9 @@
 
 package org.apache.flink.formats.avro.typeutils;
 
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.java.typeutils.runtime.KryoRegistrationSerializerConfigSnapshot;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -100,7 +100,7 @@
 	private transient Schema schema;
 
 	/** The serializer configuration snapshot, cached for efficiency. */
-	private transient AvroSchemaSerializerConfigSnapshot configSnapshot;
+	private transient AvroSchemaSerializerConfigSnapshot<T> configSnapshot;
 
 	/** The currently accessing thread, set and checked on debug level only. */
 	private transient volatile Thread currentThread;
@@ -264,20 +264,20 @@ public void copy(DataInputView source, DataOutputView target) throws IOException
 	// ------------------------------------------------------------------------
 
 	@Override
-	public TypeSerializerConfigSnapshot snapshotConfiguration() {
+	public TypeSerializerConfigSnapshot<T> snapshotConfiguration() {
 		if (configSnapshot == null) {
 			checkAvroInitialized();
-			configSnapshot = new AvroSchemaSerializerConfigSnapshot(schema.toString(false));
+			configSnapshot = new AvroSchemaSerializerConfigSnapshot<>(schema.toString(false));
 		}
 		return configSnapshot;
 	}
 
 	@Override
-	@SuppressWarnings("deprecation")
-	public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	@SuppressWarnings({"deprecation", "unchecked"})
+	public TypeSerializerSchemaCompatibility<T> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		if (configSnapshot instanceof AvroSchemaSerializerConfigSnapshot) {
 			// proper schema snapshot, can do the sophisticated schema-based compatibility check
-			final String schemaString = ((AvroSchemaSerializerConfigSnapshot) configSnapshot).getSchemaString();
+			final String schemaString = ((AvroSchemaSerializerConfigSnapshot<?>) configSnapshot).getSchemaString();
 			final Schema lastSchema = new Schema.Parser().parse(schemaString);
 
 			checkAvroInitialized();
@@ -285,18 +285,18 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() {
 					SchemaCompatibility.checkReaderWriterCompatibility(schema, lastSchema);
 
 			return compatibility.getType() == SchemaCompatibilityType.COMPATIBLE ?
-					CompatibilityResult.compatible() : CompatibilityResult.requiresMigration();
+					TypeSerializerSchemaCompatibility.compatibleAsIs() : TypeSerializerSchemaCompatibility.incompatible();
 		}
 		else if (configSnapshot instanceof AvroSerializerConfigSnapshot) {
 			// old snapshot case, just compare the type
 			// we don't need to restore any Kryo stuff, since Kryo was never used for persistence,
 			// only for object-to-object copies.
-			final AvroSerializerConfigSnapshot old = (AvroSerializerConfigSnapshot) configSnapshot;
+			final AvroSerializerConfigSnapshot<T> old = (AvroSerializerConfigSnapshot<T>) configSnapshot;
 			return type.equals(old.getTypeClass()) ?
-					CompatibilityResult.compatible() : CompatibilityResult.requiresMigration();
+					TypeSerializerSchemaCompatibility.compatibleAsIs() : TypeSerializerSchemaCompatibility.incompatible();
 		}
 		else {
-			return CompatibilityResult.requiresMigration();
+			return TypeSerializerSchemaCompatibility.incompatible();
 		}
 	}
 
@@ -419,7 +419,7 @@ private void exitExclusiveThread() {
 	/**
 	 * A config snapshot for the Avro Serializer that stores the Avro Schema to check compatibility.
 	 */
-	public static final class AvroSchemaSerializerConfigSnapshot extends TypeSerializerConfigSnapshot {
+	public static final class AvroSchemaSerializerConfigSnapshot<T> extends TypeSerializerConfigSnapshot<T> {
 
 		private String schemaString;
 
diff --git a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/BackwardsCompatibleAvroSerializer.java b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/BackwardsCompatibleAvroSerializer.java
index e5eb5d89a1f..6351b8b0420 100644
--- a/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/BackwardsCompatibleAvroSerializer.java
+++ b/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/BackwardsCompatibleAvroSerializer.java
@@ -19,9 +19,10 @@
 package org.apache.flink.formats.avro.typeutils;
 
 import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.CompatibilityUtil;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.java.typeutils.runtime.KryoRegistrationSerializerConfigSnapshot;
 import org.apache.flink.api.java.typeutils.runtime.PojoSerializer.PojoSerializerConfigSnapshot;
 import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
@@ -172,13 +173,13 @@ public String toString() {
 	// ------------------------------------------------------------------------
 
 	@Override
-	public TypeSerializerConfigSnapshot snapshotConfiguration() {
+	public TypeSerializerConfigSnapshot<T> snapshotConfiguration() {
 		// we return the configuration of the actually used serializer here
 		return serializer.snapshotConfiguration();
 	}
 
 	@Override
-	public CompatibilityResult<T> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	public TypeSerializerSchemaCompatibility<T> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		if (configSnapshot instanceof AvroSchemaSerializerConfigSnapshot ||
 				configSnapshot instanceof AvroSerializerConfigSnapshot) {
 
@@ -186,7 +187,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() {
 			checkState(serializer instanceof AvroSerializer,
 					"Serializer was changed backwards to PojoSerializer and now encounters AvroSerializer snapshot.");
 
-			return serializer.ensureCompatibility(configSnapshot);
+			return CompatibilityUtil.resolveCompatibilityResult((TypeSerializerConfigSnapshot<T>) configSnapshot, serializer);
 		}
 		else if (configSnapshot instanceof PojoSerializerConfigSnapshot) {
 			// common previous case
@@ -201,18 +202,18 @@ else if (configSnapshot instanceof PojoSerializerConfigSnapshot) {
 			final TypeSerializer<T> pojoSerializer =
 					(TypeSerializer<T>) typeInfo.createPojoSerializer(new ExecutionConfig());
 			this.serializer = pojoSerializer;
-			return serializer.ensureCompatibility(configSnapshot);
+			return CompatibilityUtil.resolveCompatibilityResult((TypeSerializerConfigSnapshot<T>) configSnapshot, serializer);
 		}
 		else if (configSnapshot instanceof KryoRegistrationSerializerConfigSnapshot) {
 			// force-kryo old case common previous case
 			// we create a new Kryo Serializer with a blank execution config.
 			// registrations are anyways picked up from the snapshot.
 			serializer = new KryoSerializer<>(type, new ExecutionConfig());
-			return serializer.ensureCompatibility(configSnapshot);
+			return CompatibilityUtil.resolveCompatibilityResult((TypeSerializerConfigSnapshot<T>) configSnapshot, serializer);
 		}
 		else {
 			// completely incompatible type, needs migration
-			return CompatibilityResult.requiresMigration();
+			return TypeSerializerSchemaCompatibility.incompatible();
 		}
 	}
 }
diff --git a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/BackwardsCompatibleAvroSerializerTest.java b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/BackwardsCompatibleAvroSerializerTest.java
index f64163656fa..1a86575c111 100644
--- a/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/BackwardsCompatibleAvroSerializerTest.java
+++ b/flink-formats/flink-avro/src/test/java/org/apache/flink/formats/avro/typeutils/BackwardsCompatibleAvroSerializerTest.java
@@ -74,7 +74,7 @@ public void testCompatibilityWithPojoSerializer() throws Exception {
 		// retrieve the old config snapshot
 
 		final TypeSerializer<User> serializer;
-		final TypeSerializerConfigSnapshot configSnapshot;
+		final TypeSerializerConfigSnapshot<User> configSnapshot;
 
 		try (InputStream in = getClass().getClassLoader().getResourceAsStream(SNAPSHOT_RESOURCE)) {
 			DataInputViewStreamWrapper inView = new DataInputViewStreamWrapper(in);
@@ -102,18 +102,18 @@ public void testCompatibilityWithPojoSerializer() throws Exception {
 		validateDeserialization(serializer);
 
 		// sanity check for the test: check that a PoJoSerializer and the original serializer work together
-		assertFalse(serializer.ensureCompatibility(configSnapshot).isRequiresMigration());
+		assertFalse(configSnapshot.resolveSchemaCompatibility(serializer).isIncompatible());
 
 		final TypeSerializer<User> newSerializer = new AvroTypeInfo<>(User.class, true).createSerializer(new ExecutionConfig());
-		assertFalse(newSerializer.ensureCompatibility(configSnapshot).isRequiresMigration());
+		assertFalse(configSnapshot.resolveSchemaCompatibility(newSerializer).isIncompatible());
 
 		// deserialize the data and make sure this still works
 		validateDeserialization(newSerializer);
 
-		TypeSerializerConfigSnapshot nextSnapshot = newSerializer.snapshotConfiguration();
+		TypeSerializerConfigSnapshot<User> nextSnapshot = newSerializer.snapshotConfiguration();
 		final TypeSerializer<User> nextSerializer = new AvroTypeInfo<>(User.class, true).createSerializer(new ExecutionConfig());
 
-		assertFalse(nextSerializer.ensureCompatibility(nextSnapshot).isRequiresMigration());
+		assertFalse(nextSnapshot.resolveSchemaCompatibility(nextSerializer).isIncompatible());
 
 		// deserialize the data and make sure this still works
 		validateDeserialization(nextSerializer);
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java b/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java
index 579e761c014..63ddea585b3 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java
+++ b/flink-java/src/test/java/org/apache/flink/api/java/io/CollectionInputFormatTest.java
@@ -21,9 +21,9 @@
 import org.apache.flink.api.common.ExecutionConfig;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.core.io.GenericInputSplit;
 import org.apache.flink.core.memory.DataInputView;
@@ -395,12 +395,12 @@ public int hashCode() {
 		}
 
 		@Override
-		public TypeSerializerConfigSnapshot snapshotConfiguration() {
+		public TypeSerializerConfigSnapshot<ElementType> snapshotConfiguration() {
 			throw new UnsupportedOperationException();
 		}
 
 		@Override
-		public CompatibilityResult<ElementType> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+		public TypeSerializerSchemaCompatibility<ElementType> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 			throw new UnsupportedOperationException();
 		}
 	}
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
index 041a01702f4..06f60d2c1b8 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java
@@ -19,13 +19,10 @@
 package org.apache.flink.cep.nfa;
 
 import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializer;
 import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.cep.nfa.compiler.NFACompiler;
@@ -863,9 +860,9 @@ private boolean checkFilterCondition(
 	 * The {@link TypeSerializerConfigSnapshot} serializer configuration to be stored with the managed state.
 	 */
 	@Deprecated
-	public static final class NFASerializerConfigSnapshot<T> extends CompositeTypeSerializerConfigSnapshot {
+	public static final class NFASerializerConfigSnapshot<T> extends CompositeTypeSerializerConfigSnapshot<MigratedNFA<T>> {
 
-		private static final int VERSION = 1;
+		private static final int VERSION = 2;
 
 		/** This empty constructor is required for deserializing the configuration. */
 		public NFASerializerConfigSnapshot() {}
@@ -881,13 +878,36 @@ public NFASerializerConfigSnapshot(
 		public int getVersion() {
 			return VERSION;
 		}
+
+		@Override
+		public int[] getCompatibleVersions() {
+			return new int[]{VERSION, 1};
+		}
+
+		@Override
+		protected boolean containsSerializers() {
+			return getReadVersion() < 2;
+		}
+
+		@SuppressWarnings("unchecked")
+		@Override
+		protected TypeSerializer<MigratedNFA<T>> restoreSerializer(TypeSerializer<?>... restoredNestedSerializers) {
+			return new NFASerializer<>(
+				(TypeSerializer<T>) restoredNestedSerializers[0],
+				(TypeSerializer<org.apache.flink.cep.nfa.SharedBuffer<T>>) restoredNestedSerializers[1]);
+		}
+
+		@Override
+		protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+			return newSerializer instanceof NFASerializer;
+		}
 	}
 
 	/**
 	 * Only for backward compatibility with <=1.5.
 	 */
 	@Deprecated
-	public static class NFASerializer<T> extends TypeSerializer<MigratedNFA<T>> {
+	public static class NFASerializer<T> extends CompositeTypeSerializer<MigratedNFA<T>> {
 
 		private static final long serialVersionUID = 2098282423980597010L;
 
@@ -905,6 +925,14 @@ public NFASerializer(TypeSerializer<T> typeSerializer) {
 		NFASerializer(
 				TypeSerializer<T> typeSerializer,
 				TypeSerializer<org.apache.flink.cep.nfa.SharedBuffer<T>> sharedBufferSerializer) {
+
+			super(
+				new NFASerializerConfigSnapshot<>(
+					Preconditions.checkNotNull(typeSerializer),
+					Preconditions.checkNotNull(sharedBufferSerializer)),
+				typeSerializer,
+				sharedBufferSerializer);
+
 			this.eventSerializer = typeSerializer;
 			this.sharedBufferSerializer = sharedBufferSerializer;
 		}
@@ -985,45 +1013,5 @@ public boolean canEqual(Object obj) {
 		public int hashCode() {
 			return 37 * sharedBufferSerializer.hashCode() + eventSerializer.hashCode();
 		}
-
-		@Override
-		public TypeSerializerConfigSnapshot snapshotConfiguration() {
-			return new NFASerializerConfigSnapshot<>(eventSerializer, sharedBufferSerializer);
-		}
-
-		@Override
-		public CompatibilityResult<MigratedNFA<T>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-			if (configSnapshot instanceof NFASerializerConfigSnapshot) {
-				List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> serializersAndConfigs =
-					((NFASerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
-
-				CompatibilityResult<T> eventCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-					serializersAndConfigs.get(0).f0,
-					UnloadableDummyTypeSerializer.class,
-					serializersAndConfigs.get(0).f1,
-					eventSerializer);
-
-				CompatibilityResult<org.apache.flink.cep.nfa.SharedBuffer<T>> sharedBufCompatResult =
-					CompatibilityUtil.resolveCompatibilityResult(
-						serializersAndConfigs.get(1).f0,
-						UnloadableDummyTypeSerializer.class,
-						serializersAndConfigs.get(1).f1,
-						sharedBufferSerializer);
-
-				if (!sharedBufCompatResult.isRequiresMigration() && !eventCompatResult.isRequiresMigration()) {
-					return CompatibilityResult.compatible();
-				} else {
-					if (eventCompatResult.getConvertDeserializer() != null &&
-						sharedBufCompatResult.getConvertDeserializer() != null) {
-						return CompatibilityResult.requiresMigration(
-							new NFASerializer<>(
-								new TypeDeserializerAdapter<>(eventCompatResult.getConvertDeserializer()),
-								new TypeDeserializerAdapter<>(sharedBufCompatResult.getConvertDeserializer())));
-					}
-				}
-			}
-
-			return CompatibilityResult.requiresMigration();
-		}
 	}
 }
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
index a4dbc00704d..013c6682c08 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/SharedBuffer.java
@@ -18,13 +18,10 @@
 
 package org.apache.flink.cep.nfa;
 
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializer;
 import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.cep.nfa.compiler.NFAStateNameHandler;
 import org.apache.flink.cep.nfa.sharedbuffer.EventId;
@@ -34,6 +31,7 @@
 import org.apache.flink.cep.nfa.sharedbuffer.SharedBufferNode;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.util.Preconditions;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -149,9 +147,10 @@ public int hashCode() {
 	/**
 	 * The {@link TypeSerializerConfigSnapshot} serializer configuration to be stored with the managed state.
 	 */
-	public static final class SharedBufferSerializerConfigSnapshot<K, V> extends CompositeTypeSerializerConfigSnapshot {
+	public static final class SharedBufferSerializerConfigSnapshot<K, V>
+			extends CompositeTypeSerializerConfigSnapshot<SharedBuffer<V>> {
 
-		private static final int VERSION = 1;
+		private static final int VERSION = 2;
 
 		/** This empty constructor is required for deserializing the configuration. */
 		public SharedBufferSerializerConfigSnapshot() {
@@ -169,12 +168,37 @@ public SharedBufferSerializerConfigSnapshot(
 		public int getVersion() {
 			return VERSION;
 		}
+
+		@Override
+		public int[] getCompatibleVersions() {
+			return new int[]{VERSION, 1};
+		}
+
+		@Override
+		protected boolean containsSerializers() {
+			return getReadVersion() < 2;
+		}
+
+		@SuppressWarnings("unchecked")
+		@Override
+		protected TypeSerializer<SharedBuffer<V>> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+			return new SharedBufferSerializer<>(
+				(TypeSerializer<K>) restoredNestedSerializers[0],
+				(TypeSerializer<V>) restoredNestedSerializers[1],
+				(TypeSerializer<DeweyNumber>) restoredNestedSerializers[2]);
+		}
+
+		@Override
+		protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+			return newSerializer instanceof SharedBufferSerializer;
+		}
 	}
 
 	/**
 	 * A {@link TypeSerializer} for the {@link SharedBuffer}.
 	 */
-	public static class SharedBufferSerializer<K, V> extends TypeSerializer<SharedBuffer<V>> {
+	public static class SharedBufferSerializer<K, V>
+			extends CompositeTypeSerializer<SharedBuffer<V>> {
 
 		private static final long serialVersionUID = -3254176794680331560L;
 
@@ -193,6 +217,15 @@ public SharedBufferSerializer(
 			final TypeSerializer<V> valueSerializer,
 			final TypeSerializer<DeweyNumber> versionSerializer) {
 
+			super(
+				new SharedBufferSerializerConfigSnapshot<>(
+					Preconditions.checkNotNull(keySerializer),
+					Preconditions.checkNotNull(valueSerializer),
+					Preconditions.checkNotNull(versionSerializer)),
+				keySerializer,
+				valueSerializer,
+				versionSerializer);
+
 			this.keySerializer = keySerializer;
 			this.valueSerializer = valueSerializer;
 			this.versionSerializer = versionSerializer;
@@ -340,57 +373,5 @@ public boolean canEqual(Object obj) {
 		public int hashCode() {
 			return 37 * keySerializer.hashCode() + valueSerializer.hashCode();
 		}
-
-		@Override
-		public TypeSerializerConfigSnapshot snapshotConfiguration() {
-			return new SharedBufferSerializerConfigSnapshot<>(
-				keySerializer,
-				valueSerializer,
-				versionSerializer);
-		}
-
-		@Override
-		public CompatibilityResult<SharedBuffer<V>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-			if (configSnapshot instanceof SharedBufferSerializerConfigSnapshot) {
-				List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> serializerConfigSnapshots =
-					((SharedBufferSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
-
-				CompatibilityResult<K> keyCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-					serializerConfigSnapshots.get(0).f0,
-					UnloadableDummyTypeSerializer.class,
-					serializerConfigSnapshots.get(0).f1,
-					keySerializer);
-
-				CompatibilityResult<V> valueCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-					serializerConfigSnapshots.get(1).f0,
-					UnloadableDummyTypeSerializer.class,
-					serializerConfigSnapshots.get(1).f1,
-					valueSerializer);
-
-				CompatibilityResult<DeweyNumber> versionCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-					serializerConfigSnapshots.get(2).f0,
-					UnloadableDummyTypeSerializer.class,
-					serializerConfigSnapshots.get(2).f1,
-					versionSerializer);
-
-				if (!keyCompatResult.isRequiresMigration() && !valueCompatResult.isRequiresMigration() &&
-					!versionCompatResult.isRequiresMigration()) {
-					return CompatibilityResult.compatible();
-				} else {
-					if (keyCompatResult.getConvertDeserializer() != null
-						&& valueCompatResult.getConvertDeserializer() != null
-						&& versionCompatResult.getConvertDeserializer() != null) {
-						return CompatibilityResult.requiresMigration(
-							new SharedBufferSerializer<>(
-								new TypeDeserializerAdapter<>(keyCompatResult.getConvertDeserializer()),
-								new TypeDeserializerAdapter<>(valueCompatResult.getConvertDeserializer()),
-								new TypeDeserializerAdapter<>(versionCompatResult.getConvertDeserializer())
-							));
-					}
-				}
-			}
-
-			return CompatibilityResult.requiresMigration();
-		}
 	}
 }
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java
index b782d8a07c6..d8a8ba721e3 100644
--- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/Lockable.java
@@ -18,13 +18,12 @@
 
 package org.apache.flink.cep.nfa.sharedbuffer;
 
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.util.Preconditions;
 
 import java.io.IOException;
 import java.util.Objects;
@@ -92,11 +91,15 @@ public int hashCode() {
 	}
 
 	/** Serializer for {@link Lockable}. */
-	public static class LockableTypeSerializer<E> extends TypeSerializer<Lockable<E>> {
+	public static class LockableTypeSerializer<E> extends CompositeTypeSerializer<Lockable<E>> {
 		private static final long serialVersionUID = 3298801058463337340L;
 		private final TypeSerializer<E> elementSerializer;
 
 		LockableTypeSerializer(TypeSerializer<E> elementSerializer) {
+			super(
+				new LockableSerializerConfigSnapshot<>(Preconditions.checkNotNull(elementSerializer)),
+				elementSerializer);
+
 			this.elementSerializer = elementSerializer;
 		}
 
@@ -179,22 +182,5 @@ public int hashCode() {
 		public boolean canEqual(Object obj) {
 			return obj.getClass().equals(LockableTypeSerializer.class);
 		}
-
-		@Override
-		public TypeSerializerConfigSnapshot snapshotConfiguration() {
-			return elementSerializer.snapshotConfiguration();
-		}
-
-		@Override
-		public CompatibilityResult<Lockable<E>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-			CompatibilityResult<E> inputComaptibilityResult = elementSerializer.ensureCompatibility(configSnapshot);
-			if (inputComaptibilityResult.isRequiresMigration()) {
-				return CompatibilityResult.requiresMigration(new LockableTypeSerializer<>(
-					new TypeDeserializerAdapter<>(inputComaptibilityResult.getConvertDeserializer()))
-				);
-			} else {
-				return CompatibilityResult.compatible();
-			}
-		}
 	}
 }
diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/LockableSerializerConfigSnapshot.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/LockableSerializerConfigSnapshot.java
new file mode 100644
index 00000000000..6bd08df637a
--- /dev/null
+++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/LockableSerializerConfigSnapshot.java
@@ -0,0 +1,53 @@
+/*
+ * 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.cep.nfa.sharedbuffer;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+
+/**
+ * A {@link TypeSerializerConfigSnapshot} for the {@link Lockable.LockableTypeSerializer}.
+ */
+@Internal
+public class LockableSerializerConfigSnapshot<E> extends CompositeTypeSerializerConfigSnapshot<Lockable<E>> {
+
+	private static final int VERSION = 1;
+
+	public LockableSerializerConfigSnapshot(TypeSerializer<E> elementSerializer) {
+		super(elementSerializer);
+	}
+
+	@Override
+	public int getVersion() {
+		return VERSION;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	protected TypeSerializer<Lockable<E>> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+		return new Lockable.LockableTypeSerializer<>((TypeSerializer<E>) restoredNestedSerializers[0]);
+	}
+
+	@Override
+	protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+		return newSerializer instanceof Lockable.LockableTypeSerializer;
+	}
+}
diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/ListViewSerializerConfigSnapshot.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/ListViewSerializerConfigSnapshot.java
new file mode 100644
index 00000000000..536522900de
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/ListViewSerializerConfigSnapshot.java
@@ -0,0 +1,55 @@
+/*
+ * 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.table.dataview;
+
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.base.ListSerializer;
+import org.apache.flink.table.api.dataview.ListView;
+
+/**
+ * A {@link TypeSerializerConfigSnapshot} for the {@link ListViewSerializer}.
+ *
+ * @param <T> the type of the list elements.
+ */
+public final class ListViewSerializerConfigSnapshot<T> extends CompositeTypeSerializerConfigSnapshot<ListView<T>> {
+
+	private static final int VERSION = 1;
+
+	public ListViewSerializerConfigSnapshot(ListSerializer<T> listSerializer) {
+		super(listSerializer);
+	}
+
+	@Override
+	public int getVersion() {
+		return VERSION;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	protected TypeSerializer<ListView<T>> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+		return new ListViewSerializer<>((ListSerializer<T>) restoredNestedSerializers[0]);
+	}
+
+	@Override
+	protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+		return newSerializer instanceof ListViewSerializer;
+	}
+}
diff --git a/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/MapViewSerializerConfigSnapshot.java b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/MapViewSerializerConfigSnapshot.java
new file mode 100644
index 00000000000..89588124462
--- /dev/null
+++ b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/MapViewSerializerConfigSnapshot.java
@@ -0,0 +1,56 @@
+/*
+ * 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.table.dataview;
+
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.base.MapSerializer;
+import org.apache.flink.table.api.dataview.MapView;
+
+/**
+ * A {@link TypeSerializerConfigSnapshot} for the {@link MapViewSerializer}.
+ *
+ * @param <K> the key type of the map entries.
+ * @param <V> the value type of the map entries.
+ */
+public class MapViewSerializerConfigSnapshot<K, V> extends CompositeTypeSerializerConfigSnapshot<MapView<K, V>> {
+
+	private static final int VERSION = 1;
+
+	public MapViewSerializerConfigSnapshot(MapSerializer<K, V> mapSerializer) {
+		super(mapSerializer);
+	}
+
+	@Override
+	public int getVersion() {
+		return VERSION;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	protected TypeSerializer<MapView<K, V>> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+		return new MapViewSerializer<>((MapSerializer<K, V>) restoredNestedSerializers[0]);
+	}
+
+	@Override
+	protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+		return newSerializer instanceof MapViewSerializer;
+	}
+}
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewSerializer.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewSerializer.scala
index a450c2ce1e5..81dac4912c9 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewSerializer.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/ListViewSerializer.scala
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeutils._
 import org.apache.flink.api.common.typeutils.base.{CollectionSerializerConfigSnapshot, ListSerializer}
 import org.apache.flink.core.memory.{DataInputView, DataOutputView}
 import org.apache.flink.table.api.dataview.ListView
+import org.apache.flink.util.Preconditions
 
 /**
   * A serializer for [[ListView]]. The serializer relies on an element
@@ -34,7 +35,9 @@ import org.apache.flink.table.api.dataview.ListView
   * @tparam T The type of element in the list.
   */
 class ListViewSerializer[T](val listSerializer: ListSerializer[T])
-  extends TypeSerializer[ListView[T]] {
+  extends CompositeTypeSerializer[ListView[T]](
+    new ListViewSerializerConfigSnapshot[T](Preconditions.checkNotNull(listSerializer)),
+    listSerializer) {
 
   override def isImmutableType: Boolean = false
 
@@ -75,37 +78,13 @@ class ListViewSerializer[T](val listSerializer: ListSerializer[T])
   override def equals(obj: Any): Boolean = canEqual(this) &&
     listSerializer.equals(obj.asInstanceOf[ListViewSerializer[_]].listSerializer)
 
-  override def snapshotConfiguration(): TypeSerializerConfigSnapshot =
-    listSerializer.snapshotConfiguration()
-
-  // copy and modified from ListSerializer.ensureCompatibility
-  override def ensureCompatibility(
-      configSnapshot: TypeSerializerConfigSnapshot): CompatibilityResult[ListView[T]] = {
-
-    configSnapshot match {
-      case snapshot: CollectionSerializerConfigSnapshot[_] =>
-        val previousListSerializerAndConfig = snapshot.getSingleNestedSerializerAndConfig
-
-        val compatResult = CompatibilityUtil.resolveCompatibilityResult(
-          previousListSerializerAndConfig.f0,
-          classOf[UnloadableDummyTypeSerializer[_]],
-          previousListSerializerAndConfig.f1,
-          listSerializer.getElementSerializer)
-
-        if (!compatResult.isRequiresMigration) {
-          CompatibilityResult.compatible[ListView[T]]
-        } else if (compatResult.getConvertDeserializer != null) {
-          CompatibilityResult.requiresMigration(
-            new ListViewSerializer[T](
-              new ListSerializer[T](
-                new TypeDeserializerAdapter[T](compatResult.getConvertDeserializer))
-            )
-          )
-        } else {
-          CompatibilityResult.requiresMigration[ListView[T]]
-        }
-
-      case _ => CompatibilityResult.requiresMigration[ListView[T]]
-    }
+  override def isComparableSnapshot(
+      configSnapshot: TypeSerializerConfigSnapshot[_]): Boolean = {
+
+    configSnapshot.isInstanceOf[ListViewSerializerConfigSnapshot[T]] ||
+      // backwards compatibility path;
+      // Flink versions older or equal to 1.5.x returns a
+      // CollectionSerializerConfigSnapshot as the snapshot
+      configSnapshot.isInstanceOf[CollectionSerializerConfigSnapshot[_, T]]
   }
 }
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewSerializer.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewSerializer.scala
index c53f10c37e5..044f2fd47b2 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewSerializer.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/dataview/MapViewSerializer.scala
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeutils._
 import org.apache.flink.api.common.typeutils.base.{MapSerializer, MapSerializerConfigSnapshot}
 import org.apache.flink.core.memory.{DataInputView, DataOutputView}
 import org.apache.flink.table.api.dataview.MapView
+import org.apache.flink.util.Preconditions
 
 /**
   * A serializer for [[MapView]]. The serializer relies on a key serializer and a value
@@ -36,7 +37,9 @@ import org.apache.flink.table.api.dataview.MapView
   * @tparam V The type of the values in the map.
   */
 class MapViewSerializer[K, V](val mapSerializer: MapSerializer[K, V])
-  extends TypeSerializer[MapView[K, V]] {
+  extends CompositeTypeSerializer[MapView[K, V]](
+    new MapViewSerializerConfigSnapshot[K, V](Preconditions.checkNotNull(mapSerializer)),
+    mapSerializer) {
 
   override def isImmutableType: Boolean = false
 
@@ -77,45 +80,13 @@ class MapViewSerializer[K, V](val mapSerializer: MapSerializer[K, V])
   override def equals(obj: Any): Boolean = canEqual(this) &&
     mapSerializer.equals(obj.asInstanceOf[MapViewSerializer[_, _]].mapSerializer)
 
-  override def snapshotConfiguration(): TypeSerializerConfigSnapshot =
-    mapSerializer.snapshotConfiguration()
-
-  // copy and modified from MapSerializer.ensureCompatibility
-  override def ensureCompatibility(configSnapshot: TypeSerializerConfigSnapshot)
-  : CompatibilityResult[MapView[K, V]] = {
-
-    configSnapshot match {
-      case snapshot: MapSerializerConfigSnapshot[_, _] =>
-        val previousKvSerializersAndConfigs = snapshot.getNestedSerializersAndConfigs
-
-        val keyCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-          previousKvSerializersAndConfigs.get(0).f0,
-          classOf[UnloadableDummyTypeSerializer[_]],
-          previousKvSerializersAndConfigs.get(0).f1,
-          mapSerializer.getKeySerializer)
-
-        val valueCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-          previousKvSerializersAndConfigs.get(1).f0,
-          classOf[UnloadableDummyTypeSerializer[_]],
-          previousKvSerializersAndConfigs.get(1).f1,
-          mapSerializer.getValueSerializer)
-
-        if (!keyCompatResult.isRequiresMigration && !valueCompatResult.isRequiresMigration) {
-          CompatibilityResult.compatible[MapView[K, V]]
-        } else if (keyCompatResult.getConvertDeserializer != null
-          && valueCompatResult.getConvertDeserializer != null) {
-          CompatibilityResult.requiresMigration(
-            new MapViewSerializer[K, V](
-              new MapSerializer[K, V](
-                new TypeDeserializerAdapter[K](keyCompatResult.getConvertDeserializer),
-                new TypeDeserializerAdapter[V](valueCompatResult.getConvertDeserializer))
-            )
-          )
-        } else {
-          CompatibilityResult.requiresMigration[MapView[K, V]]
-        }
-
-      case _ => CompatibilityResult.requiresMigration[MapView[K, V]]
-    }
+  override def isComparableSnapshot(
+      configSnapshot: TypeSerializerConfigSnapshot[_]): Boolean = {
+
+    configSnapshot.isInstanceOf[MapViewSerializerConfigSnapshot[K, V]] ||
+      // backwards compatibility path;
+      // Flink versions older or equal to 1.5.x returns a
+      // MapSerializerConfigSnapshot as the snapshot
+      configSnapshot.isInstanceOf[MapSerializerConfigSnapshot[K, V]]
   }
 }
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowSerializer.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowSerializer.scala
index caf346c3430..61b6b626155 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowSerializer.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/types/CRowSerializer.scala
@@ -80,35 +80,26 @@ class CRowSerializer(val rowSerializer: TypeSerializer[Row]) extends TypeSeriali
   // Serializer configuration snapshotting & compatibility
   // --------------------------------------------------------------------------------------------
 
-  override def snapshotConfiguration(): TypeSerializerConfigSnapshot = {
+  override def snapshotConfiguration(): TypeSerializerConfigSnapshot[CRow] = {
     new CRowSerializer.CRowSerializerConfigSnapshot(rowSerializer)
   }
 
   override def ensureCompatibility(
-      configSnapshot: TypeSerializerConfigSnapshot): CompatibilityResult[CRow] = {
+      configSnapshot: TypeSerializerConfigSnapshot[_]): TypeSerializerSchemaCompatibility[CRow] = {
 
     configSnapshot match {
       case crowSerializerConfigSnapshot: CRowSerializer.CRowSerializerConfigSnapshot =>
         val compatResult = CompatibilityUtil.resolveCompatibilityResult(
-          crowSerializerConfigSnapshot.getSingleNestedSerializerAndConfig.f0,
-          classOf[UnloadableDummyTypeSerializer[_]],
-          crowSerializerConfigSnapshot.getSingleNestedSerializerAndConfig.f1,
+          crowSerializerConfigSnapshot.getNestedSerializerConfigSnapshot(0),
           rowSerializer)
 
-        if (compatResult.isRequiresMigration) {
-          if (compatResult.getConvertDeserializer != null) {
-            CompatibilityResult.requiresMigration(
-              new CRowSerializer(
-                new TypeDeserializerAdapter(compatResult.getConvertDeserializer))
-            )
-          } else {
-            CompatibilityResult.requiresMigration()
-          }
+        if (compatResult.isIncompatible) {
+          TypeSerializerSchemaCompatibility.incompatible()
         } else {
-          CompatibilityResult.compatible()
+          TypeSerializerSchemaCompatibility.compatibleAsIs()
         }
 
-      case _ => CompatibilityResult.requiresMigration()
+      case _ => TypeSerializerSchemaCompatibility.incompatible()
     }
   }
 }
@@ -117,16 +108,37 @@ object CRowSerializer {
 
   class CRowSerializerConfigSnapshot(
       private val rowSerializer: TypeSerializer[Row])
-    extends CompositeTypeSerializerConfigSnapshot(rowSerializer) {
+    extends CompositeTypeSerializerConfigSnapshot[CRow](rowSerializer) {
 
     /** This empty nullary constructor is required for deserializing the configuration. */
     def this() = this(null)
 
     override def getVersion: Int = CRowSerializerConfigSnapshot.VERSION
+
+    override def restoreSerializer(
+        restoredNestedSerializers: TypeSerializer[_]*
+      ): TypeSerializer[CRow] = {
+
+      new CRowSerializer(
+        restoredNestedSerializers(0).asInstanceOf[TypeSerializer[Row]])
+    }
+
+    override def containsSerializers(): Boolean = {
+      getReadVersion < 2
+    }
+
+    override def isRecognizableSerializer(
+        newSerializer: TypeSerializer[_]): Boolean = {
+      newSerializer.isInstanceOf[CRowSerializer]
+    }
+
+    override def getCompatibleVersions: Array[Int] = {
+      Array(CRowSerializerConfigSnapshot.VERSION, 1)
+    }
   }
 
   object CRowSerializerConfigSnapshot {
-    val VERSION = 1
+    val VERSION = 2
   }
 
 }
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ArrayListSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ArrayListSerializer.java
index 3c4f4b0a269..c60a3b402b4 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ArrayListSerializer.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ArrayListSerializer.java
@@ -17,28 +17,31 @@
  */
 package org.apache.flink.runtime.state;
 
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.api.common.typeutils.base.CollectionSerializerConfigSnapshot;
-import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 
 import java.io.IOException;
 import java.util.ArrayList;
 
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
 @SuppressWarnings("ForLoopReplaceableByForEach")
-final public class ArrayListSerializer<T> extends TypeSerializer<ArrayList<T>> {
+final public class ArrayListSerializer<T> extends CompositeTypeSerializer<ArrayList<T>> {
 
 	private static final long serialVersionUID = 1119562170939152304L;
 
 	private final TypeSerializer<T> elementSerializer;
 
 	public ArrayListSerializer(TypeSerializer<T> elementSerializer) {
+
+		super(
+			new ArrayListSerializerConfigSnapshot<>(checkNotNull(elementSerializer)),
+			elementSerializer);
+
 		this.elementSerializer = elementSerializer;
 	}
 
@@ -140,35 +143,9 @@ public int hashCode() {
 		return elementSerializer.hashCode();
 	}
 
-	// --------------------------------------------------------------------------------------------
-	// Serializer configuration snapshotting & compatibility
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public TypeSerializerConfigSnapshot snapshotConfiguration() {
-		return new CollectionSerializerConfigSnapshot<>(elementSerializer);
-	}
-
 	@Override
-	public CompatibilityResult<ArrayList<T>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-		if (configSnapshot instanceof CollectionSerializerConfigSnapshot) {
-			Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> previousElemSerializerAndConfig =
-				((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig();
-
-			CompatibilityResult<T> compatResult = CompatibilityUtil.resolveCompatibilityResult(
-					previousElemSerializerAndConfig.f0,
-					UnloadableDummyTypeSerializer.class,
-					previousElemSerializerAndConfig.f1,
-					elementSerializer);
-
-			if (!compatResult.isRequiresMigration()) {
-				return CompatibilityResult.compatible();
-			} else if (compatResult.getConvertDeserializer() != null) {
-				return CompatibilityResult.requiresMigration(
-					new ArrayListSerializer<>(new TypeDeserializerAdapter<>(compatResult.getConvertDeserializer())));
-			}
-		}
-
-		return CompatibilityResult.requiresMigration();
+	protected boolean isComparableSnapshot(TypeSerializerConfigSnapshot<?> configSnapshot) {
+		return configSnapshot instanceof ArrayListSerializerConfigSnapshot
+			|| configSnapshot instanceof CollectionSerializerConfigSnapshot;
 	}
 }
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ArrayListSerializerConfigSnapshot.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ArrayListSerializerConfigSnapshot.java
new file mode 100644
index 00000000000..d32146bc637
--- /dev/null
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ArrayListSerializerConfigSnapshot.java
@@ -0,0 +1,53 @@
+/*
+ * 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.runtime.state;
+
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+
+import java.util.ArrayList;
+
+/**
+ * A {@link TypeSerializerConfigSnapshot} for the {@link ArrayListSerializer}.
+ */
+public class ArrayListSerializerConfigSnapshot<T> extends CompositeTypeSerializerConfigSnapshot<ArrayList<T>> {
+
+	private final int VERSION = 1;
+
+	public ArrayListSerializerConfigSnapshot(TypeSerializer<T> elementSerializer) {
+		super(elementSerializer);
+	}
+
+	@Override
+	public int getVersion() {
+		return VERSION;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	protected TypeSerializer<ArrayList<T>> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+		return new ArrayListSerializer<>((TypeSerializer<T>) restoredNestedSerializers[0]);
+	}
+
+	@Override
+	protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+		return newSerializer instanceof ArrayListSerializer;
+	}
+}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
index a2e49cb179d..4f1f2641a1c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/DefaultOperatorStateBackend.java
@@ -24,7 +24,7 @@
 import org.apache.flink.api.common.state.ListState;
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.state.MapStateDescriptor;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.CompatibilityUtil;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
@@ -225,19 +225,15 @@ public void dispose() {
 					(RegisteredBroadcastBackendStateMetaInfo.Snapshot<K, V>) restoredBroadcastStateMetaInfos.get(name);
 
 			// check compatibility to determine if state migration is required
-			CompatibilityResult<K> keyCompatibility = CompatibilityUtil.resolveCompatibilityResult(
-					restoredMetaInfo.getKeySerializer(),
-					UnloadableDummyTypeSerializer.class,
+			TypeSerializerSchemaCompatibility<K> keyCompatibility = CompatibilityUtil.resolveCompatibilityResult(
 					restoredMetaInfo.getKeySerializerConfigSnapshot(),
 					broadcastStateKeySerializer);
 
-			CompatibilityResult<V> valueCompatibility = CompatibilityUtil.resolveCompatibilityResult(
-					restoredMetaInfo.getValueSerializer(),
-					UnloadableDummyTypeSerializer.class,
+			TypeSerializerSchemaCompatibility<V> valueCompatibility = CompatibilityUtil.resolveCompatibilityResult(
 					restoredMetaInfo.getValueSerializerConfigSnapshot(),
 					broadcastStateValueSerializer);
 
-			if (!keyCompatibility.isRequiresMigration() && !valueCompatibility.isRequiresMigration()) {
+			if (!keyCompatibility.isIncompatible() && !valueCompatibility.isIncompatible()) {
 				// new serializer is compatible; use it to replace the old serializer
 				broadcastState.setStateMetaInfo(
 						new RegisteredBroadcastBackendStateMetaInfo<>(
@@ -503,8 +499,10 @@ public void restore(Collection<OperatorStateHandle> restoreSnapshots) throws Exc
 				// Recreate all PartitionableListStates from the meta info
 				for (RegisteredOperatorBackendStateMetaInfo.Snapshot<?> restoredMetaInfo : restoredOperatorMetaInfoSnapshots) {
 
-					if (restoredMetaInfo.getPartitionStateSerializer() == null ||
-							restoredMetaInfo.getPartitionStateSerializer() instanceof UnloadableDummyTypeSerializer) {
+					TypeSerializer<?> restoredPartitionStateSerializer =
+						restoredMetaInfo.getPartitionStateSerializerConfigSnapshot().restoreSerializer();
+
+					if (restoredPartitionStateSerializer instanceof UnloadableDummyTypeSerializer) {
 
 						// must fail now if the previous serializer cannot be restored because there is no serializer
 						// capable of reading previous state
@@ -525,7 +523,7 @@ public void restore(Collection<OperatorStateHandle> restoreSnapshots) throws Exc
 						listState = new PartitionableListState<>(
 								new RegisteredOperatorBackendStateMetaInfo<>(
 										restoredMetaInfo.getName(),
-										restoredMetaInfo.getPartitionStateSerializer(),
+										restoredPartitionStateSerializer,
 										restoredMetaInfo.getAssignmentMode()));
 
 						registeredOperatorStates.put(listState.getStateMetaInfo().getName(), listState);
@@ -540,9 +538,14 @@ public void restore(Collection<OperatorStateHandle> restoreSnapshots) throws Exc
 
 				for (RegisteredBroadcastBackendStateMetaInfo.Snapshot<? ,?> restoredMetaInfo : restoredBroadcastMetaInfoSnapshots) {
 
-					if (restoredMetaInfo.getKeySerializer() == null || restoredMetaInfo.getValueSerializer() == null ||
-							restoredMetaInfo.getKeySerializer() instanceof UnloadableDummyTypeSerializer ||
-							restoredMetaInfo.getValueSerializer() instanceof UnloadableDummyTypeSerializer) {
+					TypeSerializer<?> restoredKeySerializer =
+						restoredMetaInfo.getKeySerializerConfigSnapshot().restoreSerializer();
+
+					TypeSerializer<?> restoredValueSerializer =
+						restoredMetaInfo.getValueSerializerConfigSnapshot().restoreSerializer();
+
+					if (restoredKeySerializer instanceof UnloadableDummyTypeSerializer ||
+							restoredValueSerializer instanceof UnloadableDummyTypeSerializer) {
 
 						// must fail now if the previous serializer cannot be restored because there is no serializer
 						// capable of reading previous state
@@ -564,8 +567,8 @@ public void restore(Collection<OperatorStateHandle> restoreSnapshots) throws Exc
 								new RegisteredBroadcastBackendStateMetaInfo<>(
 										restoredMetaInfo.getName(),
 										restoredMetaInfo.getAssignmentMode(),
-										restoredMetaInfo.getKeySerializer(),
-										restoredMetaInfo.getValueSerializer()));
+										restoredKeySerializer,
+										restoredValueSerializer));
 
 						registeredBroadcastStates.put(broadcastState.getStateMetaInfo().getName(), broadcastState);
 					} else {
@@ -758,13 +761,11 @@ public void addAll(List<S> values) throws Exception {
 
 			// check compatibility to determine if state migration is required
 			TypeSerializer<S> newPartitionStateSerializer = partitionStateSerializer.duplicate();
-			CompatibilityResult<S> stateCompatibility = CompatibilityUtil.resolveCompatibilityResult(
-					restoredMetaInfo.getPartitionStateSerializer(),
-					UnloadableDummyTypeSerializer.class,
+			TypeSerializerSchemaCompatibility<S> stateCompatibility = CompatibilityUtil.resolveCompatibilityResult(
 					restoredMetaInfo.getPartitionStateSerializerConfigSnapshot(),
 					newPartitionStateSerializer);
 
-			if (!stateCompatibility.isRequiresMigration()) {
+			if (!stateCompatibility.isIncompatible()) {
 				// new serializer is compatible; use it to replace the old serializer
 				partitionableListState.setStateMetaInfo(
 					new RegisteredOperatorBackendStateMetaInfo<>(name, newPartitionStateSerializer, mode));
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedBackendSerializationProxy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedBackendSerializationProxy.java
index f32e646e6ef..10d681e3f2f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedBackendSerializationProxy.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedBackendSerializationProxy.java
@@ -18,8 +18,10 @@
 
 package org.apache.flink.runtime.state;
 
+import org.apache.flink.api.common.typeutils.BackwardsCompatibleConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshotSerializationUtil;
 import org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil;
 import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
@@ -30,7 +32,6 @@
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
 /**
@@ -39,7 +40,7 @@
  */
 public class KeyedBackendSerializationProxy<K> extends VersionedIOReadableWritable {
 
-	public static final int VERSION = 4;
+	public static final int VERSION = 5;
 
 	//TODO allow for more (user defined) compression formats + backwards compatibility story.
 	/** This specifies if we use a compressed format write the key-groups */
@@ -48,8 +49,7 @@
 	/** This specifies whether or not to use dummy {@link UnloadableDummyTypeSerializer} when serializers cannot be read. */
 	private boolean isSerializerPresenceRequired;
 
-	private TypeSerializer<K> keySerializer;
-	private TypeSerializerConfigSnapshot keySerializerConfigSnapshot;
+	private TypeSerializerConfigSnapshot<K> keySerializerConfigSnapshot;
 
 	private List<RegisteredKeyedBackendStateMetaInfo.Snapshot<?, ?>> stateMetaInfoSnapshots;
 
@@ -67,7 +67,7 @@ public KeyedBackendSerializationProxy(
 
 		this.usingKeyGroupCompression = compression;
 
-		this.keySerializer = Preconditions.checkNotNull(keySerializer);
+		Preconditions.checkNotNull(keySerializer);
 		this.keySerializerConfigSnapshot = Preconditions.checkNotNull(keySerializer.snapshotConfiguration());
 
 		Preconditions.checkNotNull(stateMetaInfoSnapshots);
@@ -79,10 +79,6 @@ public KeyedBackendSerializationProxy(
 		return stateMetaInfoSnapshots;
 	}
 
-	public TypeSerializer<K> getKeySerializer() {
-		return keySerializer;
-	}
-
 	public TypeSerializerConfigSnapshot getKeySerializerConfigSnapshot() {
 		return keySerializerConfigSnapshot;
 	}
@@ -98,8 +94,8 @@ public int getVersion() {
 
 	@Override
 	public int[] getCompatibleVersions() {
-		// we are compatible with version 3 (Flink 1.3.x) and version 1 & 2 (Flink 1.2.x)
-		return new int[] {VERSION, 3, 2, 1};
+		// we are compatible with version 4 (Flink 1.5.x), version 3 (Flink 1.3.x+) and version 1 & 2 (Flink 1.2.x)
+		return new int[] {VERSION, 4, 3, 2, 1};
 	}
 
 	@Override
@@ -109,11 +105,7 @@ public void write(DataOutputView out) throws IOException {
 		// write the compression format used to write each key-group
 		out.writeBoolean(usingKeyGroupCompression);
 
-		// write in a way to be fault tolerant of read failures when deserializing the key serializer
-		TypeSerializerSerializationUtil.writeSerializersAndConfigsWithResilience(
-				out,
-				Collections.singletonList(
-					new Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>(keySerializer, keySerializerConfigSnapshot)));
+		TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(out, keySerializerConfigSnapshot);
 
 		// write individual registered keyed state metainfos
 		out.writeShort(stateMetaInfoSnapshots.size());
@@ -137,19 +129,23 @@ public void read(DataInputView in) throws IOException {
 			usingKeyGroupCompression = false;
 		}
 
-		// only starting from version 3, we have the key serializer and its config snapshot written
-		if (readVersion >= 3) {
+		if (readVersion >= 5) {
+			this.keySerializerConfigSnapshot = TypeSerializerConfigSnapshotSerializationUtil
+				.readSerializerConfigSnapshot(in, userCodeClassLoader);
+		} else if (readVersion >= 3) {
+			// versions 3 and 4 still write serializers in checkpoints
 			Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> keySerializerAndConfig =
-					TypeSerializerSerializationUtil.readSerializersAndConfigsWithResilience(in, userCodeClassLoader).get(0);
-			this.keySerializer = (TypeSerializer<K>) keySerializerAndConfig.f0;
-			this.keySerializerConfigSnapshot = keySerializerAndConfig.f1;
+				TypeSerializerSerializationUtil.readSerializersAndConfigsWithResilience(in, userCodeClassLoader).get(0);
+
+			this.keySerializerConfigSnapshot = new BackwardsCompatibleConfigSnapshot<K>(
+				keySerializerAndConfig.f1,
+				(TypeSerializer<K>) keySerializerAndConfig.f0);
 		} else {
-			this.keySerializer = TypeSerializerSerializationUtil.tryReadSerializer(in, userCodeClassLoader, true);
-			this.keySerializerConfigSnapshot = null;
-		}
+			// read through the serializer
+			TypeSerializerSerializationUtil.tryReadSerializer(in, userCodeClassLoader, true);
 
-		if (isSerializerPresenceRequired) {
-			checkSerializerPresence(keySerializer);
+			// there is not config snapshot in either older versions (<= 2)
+			this.keySerializerConfigSnapshot = null;
 		}
 
 		int numKvStates = in.readShort();
@@ -160,17 +156,17 @@ public void read(DataInputView in) throws IOException {
 				.readStateMetaInfo(in);
 
 			if (isSerializerPresenceRequired) {
-				checkSerializerPresence(snapshot.getNamespaceSerializer());
-				checkSerializerPresence(snapshot.getStateSerializer());
+				checkSerializerPresence(snapshot.getNamespaceSerializerConfigSnapshot());
+				checkSerializerPresence(snapshot.getStateSerializerConfigSnapshot());
 			}
 			stateMetaInfoSnapshots.add(snapshot);
 		}
 	}
 
-	private void checkSerializerPresence(TypeSerializer<?> serializer) throws IOException {
-		if (serializer instanceof UnloadableDummyTypeSerializer) {
+	private void checkSerializerPresence(TypeSerializerConfigSnapshot<?> snapshot) throws IOException {
+		if (snapshot.restoreSerializer() instanceof UnloadableDummyTypeSerializer) {
 			throw new IOException("Unable to restore keyed state, because a previous serializer" +
-				" of the keyed state is not present The serializer could have been removed from the classpath, " +
+				" of the keyed state is not present. The serializer could have been removed from the classpath, " +
 				" or its implementation have changed and could not be loaded. This is a temporary restriction that will" +
 				" be fixed in future versions.");
 		}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedBackendStateMetaInfoSnapshotReaderWriters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedBackendStateMetaInfoSnapshotReaderWriters.java
index f99234180c9..5277de7fdcf 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedBackendStateMetaInfoSnapshotReaderWriters.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedBackendStateMetaInfoSnapshotReaderWriters.java
@@ -19,8 +19,10 @@
 package org.apache.flink.runtime.state;
 
 import org.apache.flink.api.common.state.StateDescriptor;
+import org.apache.flink.api.common.typeutils.BackwardsCompatibleConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshotSerializationUtil;
 import org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.memory.DataInputView;
@@ -41,6 +43,9 @@
 	//  Writers
 	//   - v1: Flink 1.2.x
 	//   - v2: Flink 1.3.x
+	//   - v3: Flink 1.4.x
+	//   - v4: Flink 1.5.x
+	//   - v5: Flink 1.6.x
 	// -------------------------------------------------------------------------------
 
 	public static <N, S> KeyedBackendStateMetaInfoWriter getWriterForVersion(
@@ -52,9 +57,12 @@
 				return new KeyedBackendStateMetaInfoWriterV1V2<>(stateMetaInfo);
 
 			case 3:
+			case 4:
+				return new KeyedBackendStateMetaInfoWriterV3V4<>(stateMetaInfo);
+
 			// current version
 			case KeyedBackendSerializationProxy.VERSION:
-				return new KeyedBackendStateMetaInfoWriterV3<>(stateMetaInfo);
+				return new KeyedBackendStateMetaInfoWriterV5<>(stateMetaInfo);
 
 			default:
 				// guard for future
@@ -88,14 +96,16 @@ public void writeStateMetaInfo(DataOutputView out) throws IOException {
 			out.writeInt(stateMetaInfo.getStateType().ordinal());
 			out.writeUTF(stateMetaInfo.getName());
 
-			TypeSerializerSerializationUtil.writeSerializer(out, stateMetaInfo.getNamespaceSerializer());
-			TypeSerializerSerializationUtil.writeSerializer(out, stateMetaInfo.getStateSerializer());
+			// state meta info snapshots no longer contain serializers, so we use null just as a placeholder;
+			// this is maintained here to keep track of previous versions' serialization formats
+			TypeSerializerSerializationUtil.writeSerializer(out, null);
+			TypeSerializerSerializationUtil.writeSerializer(out, null);
 		}
 	}
 
-	static class KeyedBackendStateMetaInfoWriterV3<N, S> extends AbstractKeyedBackendStateMetaInfoWriter<N, S> {
+	static class KeyedBackendStateMetaInfoWriterV3V4<N, S> extends AbstractKeyedBackendStateMetaInfoWriter<N, S> {
 
-		public KeyedBackendStateMetaInfoWriterV3(RegisteredKeyedBackendStateMetaInfo.Snapshot<N, S> stateMetaInfo) {
+		public KeyedBackendStateMetaInfoWriterV3V4(RegisteredKeyedBackendStateMetaInfo.Snapshot<N, S> stateMetaInfo) {
 			super(stateMetaInfo);
 		}
 
@@ -107,11 +117,31 @@ public void writeStateMetaInfo(DataOutputView out) throws IOException {
 			// write in a way that allows us to be fault-tolerant and skip blocks in the case of java serialization failures
 			TypeSerializerSerializationUtil.writeSerializersAndConfigsWithResilience(
 				out,
+				// state meta info snapshots no longer contain serializers, so we use null just as a placeholder;
+				// this is maintained here to keep track of previous versions' serialization formats
 				Arrays.asList(
-					new Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>(
-						stateMetaInfo.getNamespaceSerializer(), stateMetaInfo.getNamespaceSerializerConfigSnapshot()),
-					new Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>(
-						stateMetaInfo.getStateSerializer(), stateMetaInfo.getStateSerializerConfigSnapshot())));
+					new Tuple2<>(
+						null, stateMetaInfo.getNamespaceSerializerConfigSnapshot()),
+					new Tuple2<>(
+						null, stateMetaInfo.getStateSerializerConfigSnapshot())));
+		}
+	}
+
+	static class KeyedBackendStateMetaInfoWriterV5<N, S> extends AbstractKeyedBackendStateMetaInfoWriter<N, S> {
+
+		public KeyedBackendStateMetaInfoWriterV5(RegisteredKeyedBackendStateMetaInfo.Snapshot<N, S> stateMetaInfo) {
+			super(stateMetaInfo);
+		}
+
+		@Override
+		public void writeStateMetaInfo(DataOutputView out) throws IOException {
+			out.writeInt(stateMetaInfo.getStateType().ordinal());
+			out.writeUTF(stateMetaInfo.getName());
+
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(
+				out, stateMetaInfo.getNamespaceSerializerConfigSnapshot());
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(
+				out, stateMetaInfo.getStateSerializerConfigSnapshot());
 		}
 	}
 
@@ -120,6 +150,9 @@ public void writeStateMetaInfo(DataOutputView out) throws IOException {
 	//  Readers
 	//   - v1: Flink 1.2.x
 	//   - v2: Flink 1.3.x
+	//   - v3: Flink 1.4.x
+	//   - v4: Flink 1.5.x
+	//   - v5: Flink 1.6.x
 	// -------------------------------------------------------------------------------
 
 	public static KeyedBackendStateMetaInfoReader getReaderForVersion(
@@ -130,10 +163,13 @@ public static KeyedBackendStateMetaInfoReader getReaderForVersion(
 			case 2:
 				return new KeyedBackendStateMetaInfoReaderV1V2<>(userCodeClassLoader);
 
-			// current version
 			case 3:
+			case 4:
+				return new KeyedBackendStateMetaInfoReaderV3V4<>(userCodeClassLoader);
+
+			// current version
 			case KeyedBackendSerializationProxy.VERSION:
-				return new KeyedBackendStateMetaInfoReaderV3<>(userCodeClassLoader);
+				return new KeyedBackendStateMetaInfoReaderV5<>(userCodeClassLoader);
 
 			default:
 				// guard for future
@@ -170,21 +206,27 @@ public KeyedBackendStateMetaInfoReaderV1V2(ClassLoader userCodeClassLoader) {
 			metaInfo.setStateType(StateDescriptor.Type.values()[in.readInt()]);
 			metaInfo.setName(in.readUTF());
 
-			metaInfo.setNamespaceSerializer(TypeSerializerSerializationUtil.<N>tryReadSerializer(in, userCodeClassLoader, true));
-			metaInfo.setStateSerializer(TypeSerializerSerializationUtil.<S>tryReadSerializer(in, userCodeClassLoader, true));
+			final TypeSerializerSerializationUtil.TypeSerializerSerializationProxy<N> namespaceSerializerProxy =
+				new TypeSerializerSerializationUtil.TypeSerializerSerializationProxy<>(userCodeClassLoader);
+			namespaceSerializerProxy.read(in);
 
-			// older versions do not contain the configuration snapshot
-			metaInfo.setNamespaceSerializerConfigSnapshot(null);
-			metaInfo.setStateSerializerConfigSnapshot(null);
+			final TypeSerializerSerializationUtil.TypeSerializerSerializationProxy<S> stateSerializerProxy =
+				new TypeSerializerSerializationUtil.TypeSerializerSerializationProxy<>(userCodeClassLoader);
+			stateSerializerProxy.read(in);
+
+			// older versions do not contain the configuration snapshot;
+			// we deserialize the written serializers, and then simply take a snapshot of them now
+			metaInfo.setNamespaceSerializerConfigSnapshot(namespaceSerializerProxy.getTypeSerializer().snapshotConfiguration());
+			metaInfo.setStateSerializerConfigSnapshot(stateSerializerProxy.getTypeSerializer().snapshotConfiguration());
 
 			return metaInfo;
 		}
 	}
 
 	@SuppressWarnings("unchecked")
-	static class KeyedBackendStateMetaInfoReaderV3<N, S> extends AbstractKeyedBackendStateMetaInfoReader {
+	static class KeyedBackendStateMetaInfoReaderV3V4<N, S> extends AbstractKeyedBackendStateMetaInfoReader {
 
-		public KeyedBackendStateMetaInfoReaderV3(ClassLoader userCodeClassLoader) {
+		public KeyedBackendStateMetaInfoReaderV3V4(ClassLoader userCodeClassLoader) {
 			super(userCodeClassLoader);
 		}
 
@@ -199,11 +241,34 @@ public KeyedBackendStateMetaInfoReaderV3(ClassLoader userCodeClassLoader) {
 			List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> serializersAndConfigs =
 				TypeSerializerSerializationUtil.readSerializersAndConfigsWithResilience(in, userCodeClassLoader);
 
-			metaInfo.setNamespaceSerializer((TypeSerializer<N>) serializersAndConfigs.get(0).f0);
-			metaInfo.setNamespaceSerializerConfigSnapshot(serializersAndConfigs.get(0).f1);
+			metaInfo.setNamespaceSerializerConfigSnapshot(
+				new BackwardsCompatibleConfigSnapshot(serializersAndConfigs.get(0).f1, serializersAndConfigs.get(0).f0));
+			metaInfo.setStateSerializerConfigSnapshot(
+				new BackwardsCompatibleConfigSnapshot(serializersAndConfigs.get(1).f1, serializersAndConfigs.get(1).f0));
+
+			return metaInfo;
+		}
+	}
+
+	static class KeyedBackendStateMetaInfoReaderV5<N, S> extends AbstractKeyedBackendStateMetaInfoReader {
+
+		public KeyedBackendStateMetaInfoReaderV5(ClassLoader userCodeClassLoader) {
+			super(userCodeClassLoader);
+		}
+
+		@SuppressWarnings("unchecked")
+		@Override
+		public RegisteredKeyedBackendStateMetaInfo.Snapshot readStateMetaInfo(DataInputView in) throws IOException {
+			RegisteredKeyedBackendStateMetaInfo.Snapshot<N, S> metaInfo =
+				new RegisteredKeyedBackendStateMetaInfo.Snapshot<>();
+
+			metaInfo.setStateType(StateDescriptor.Type.values()[in.readInt()]);
+			metaInfo.setName(in.readUTF());
 
-			metaInfo.setStateSerializer((TypeSerializer<S>) serializersAndConfigs.get(1).f0);
-			metaInfo.setStateSerializerConfigSnapshot(serializersAndConfigs.get(1).f1);
+			metaInfo.setNamespaceSerializerConfigSnapshot(TypeSerializerConfigSnapshotSerializationUtil
+				.readSerializerConfigSnapshot(in, userCodeClassLoader));
+			metaInfo.setStateSerializerConfigSnapshot(TypeSerializerConfigSnapshotSerializationUtil
+				.readSerializerConfigSnapshot(in, userCodeClassLoader));
 
 			return metaInfo;
 		}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorBackendSerializationProxy.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorBackendSerializationProxy.java
index e73f83a9e1d..d1767d50e8e 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorBackendSerializationProxy.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorBackendSerializationProxy.java
@@ -33,7 +33,7 @@
  */
 public class OperatorBackendSerializationProxy extends VersionedIOReadableWritable {
 
-	public static final int VERSION = 3;
+	public static final int VERSION = 4;
 
 	private List<RegisteredOperatorBackendStateMetaInfo.Snapshot<?>> operatorStateMetaInfoSnapshots;
 	private List<RegisteredBroadcastBackendStateMetaInfo.Snapshot<?, ?>> broadcastStateMetaInfoSnapshots;
@@ -62,8 +62,8 @@ public int getVersion() {
 
 	@Override
 	public int[] getCompatibleVersions() {
-		// we are compatible with version 3 (Flink 1.5.x), 2 (Flink 1.4.x, Flink 1.3.x) and version 1 (Flink 1.2.x)
-		return new int[] {VERSION, 2, 1};
+		// we are compatible with version 4 (Flink 1.6.x), 3 (Flink 1.5.x), 2 (Flink 1.4.x, Flink 1.3.x) and version 1 (Flink 1.2.x)
+		return new int[] {VERSION, 3, 2, 1};
 	}
 
 	@Override
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorBackendStateMetaInfoSnapshotReaderWriters.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorBackendStateMetaInfoSnapshotReaderWriters.java
index fafd5423e01..9a0329298de 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorBackendStateMetaInfoSnapshotReaderWriters.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorBackendStateMetaInfoSnapshotReaderWriters.java
@@ -18,8 +18,10 @@
 
 package org.apache.flink.runtime.state;
 
+import org.apache.flink.api.common.typeutils.BackwardsCompatibleConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshotSerializationUtil;
 import org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream;
@@ -27,8 +29,6 @@
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.util.InstantiationUtil;
 import org.apache.flink.util.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
 import java.util.Arrays;
@@ -41,13 +41,12 @@
  */
 public class OperatorBackendStateMetaInfoSnapshotReaderWriters {
 
-	private static final Logger LOG = LoggerFactory.getLogger(OperatorBackendStateMetaInfoSnapshotReaderWriters.class);
-
 	// -------------------------------------------------------------------------------
 	//  Writers
 	//   - v1: Flink 1.2.x
 	//   - v2: Flink 1.3.x
 	//   - v3: Flink 1.5.x
+	//   - v4: Flink 1.6.x
 	// -------------------------------------------------------------------------------
 
 	public static <S> OperatorBackendStateMetaInfoWriter getOperatorStateWriterForVersion(
@@ -59,9 +58,12 @@
 
 			// current version
 			case 2:
-			case OperatorBackendSerializationProxy.VERSION:
-				return new OperatorBackendStateMetaInfoWriterV2<>(stateMetaInfo);
+			case 3:
+				return new OperatorBackendStateMetaInfoWriterV2V3<>(stateMetaInfo);
 
+			// current version
+			case OperatorBackendSerializationProxy.VERSION:
+				return new OperatorBackendStateMetaInfoWriterV4<>(stateMetaInfo);
 			default:
 				// guard for future
 				throw new IllegalStateException(
@@ -74,9 +76,12 @@
 			final RegisteredBroadcastBackendStateMetaInfo.Snapshot<K, V> broadcastStateMetaInfo) {
 
 		switch (version) {
+			case 3:
+				return new BroadcastStateMetaInfoWriterV3<>(broadcastStateMetaInfo);
+
 			// current version
 			case OperatorBackendSerializationProxy.VERSION:
-				return new BroadcastStateMetaInfoWriterV3<>(broadcastStateMetaInfo);
+				return new BroadcastStateMetaInfoWriterV4<>(broadcastStateMetaInfo);
 
 			default:
 				// guard for future
@@ -123,13 +128,16 @@ public OperatorBackendStateMetaInfoWriterV1(RegisteredOperatorBackendStateMetaIn
 		public void writeOperatorStateMetaInfo(DataOutputView out) throws IOException {
 			out.writeUTF(stateMetaInfo.getName());
 			out.writeByte(stateMetaInfo.getAssignmentMode().ordinal());
-			TypeSerializerSerializationUtil.writeSerializer(out, stateMetaInfo.getPartitionStateSerializer());
+
+			// state meta info snapshots no longer contain serializers, so we use null just as a placeholder;
+			// this is maintained here to keep track of previous versions' serialization formats
+			TypeSerializerSerializationUtil.writeSerializer(out, null);
 		}
 	}
 
-	public static class OperatorBackendStateMetaInfoWriterV2<S> extends AbstractOperatorBackendStateMetaInfoWriter<S> {
+	public static class OperatorBackendStateMetaInfoWriterV2V3<S> extends AbstractOperatorBackendStateMetaInfoWriter<S> {
 
-		public OperatorBackendStateMetaInfoWriterV2(RegisteredOperatorBackendStateMetaInfo.Snapshot<S> stateMetaInfo) {
+		public OperatorBackendStateMetaInfoWriterV2V3(RegisteredOperatorBackendStateMetaInfo.Snapshot<S> stateMetaInfo) {
 			super(stateMetaInfo);
 		}
 
@@ -141,12 +149,31 @@ public void writeOperatorStateMetaInfo(DataOutputView out) throws IOException {
 			// write in a way that allows us to be fault-tolerant and skip blocks in the case of java serialization failures
 			TypeSerializerSerializationUtil.writeSerializersAndConfigsWithResilience(
 				out,
-				Collections.singletonList(new Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>(
-					stateMetaInfo.getPartitionStateSerializer(),
+				Collections.singletonList(new Tuple2<>(
+					// state meta info snapshots no longer contain serializers, so we use null just as a placeholder;
+					// this is maintained here to keep track of previous versions' serialization formats
+					null,
 					stateMetaInfo.getPartitionStateSerializerConfigSnapshot())));
 		}
 	}
 
+	public static class OperatorBackendStateMetaInfoWriterV4<S> extends AbstractOperatorBackendStateMetaInfoWriter<S> {
+
+		public OperatorBackendStateMetaInfoWriterV4(RegisteredOperatorBackendStateMetaInfo.Snapshot<S> stateMetaInfo) {
+			super(stateMetaInfo);
+		}
+
+		@Override
+		public void writeOperatorStateMetaInfo(DataOutputView out) throws IOException {
+			out.writeUTF(stateMetaInfo.getName());
+			out.writeByte(stateMetaInfo.getAssignmentMode().ordinal());
+
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(
+				out,
+				stateMetaInfo.getPartitionStateSerializerConfigSnapshot());
+		}
+	}
+
 	public static class BroadcastStateMetaInfoWriterV3<K, V> extends AbstractBroadcastStateMetaInfoWriter<K, V> {
 
 		public BroadcastStateMetaInfoWriterV3(
@@ -162,13 +189,15 @@ public void writeBroadcastStateMetaInfo(final DataOutputView out) throws IOExcep
 			// write in a way that allows us to be fault-tolerant and skip blocks in the case of java serialization failures
 			TypeSerializerSerializationUtil.writeSerializersAndConfigsWithResilience(
 					out,
+					// state meta info snapshots no longer contain serializers, so we use null just as a placeholder;
+					// this is maintained here to keep track of previous versions' serialization formats
 					Arrays.asList(
 							Tuple2.of(
-									broadcastStateMetaInfo.getKeySerializer(),
+									null,
 									broadcastStateMetaInfo.getKeySerializerConfigSnapshot()
 							),
 							Tuple2.of(
-									broadcastStateMetaInfo.getValueSerializer(),
+									null,
 									broadcastStateMetaInfo.getValueSerializerConfigSnapshot()
 							)
 					)
@@ -176,11 +205,34 @@ public void writeBroadcastStateMetaInfo(final DataOutputView out) throws IOExcep
 		}
 	}
 
+	public static class BroadcastStateMetaInfoWriterV4<K, V> extends AbstractBroadcastStateMetaInfoWriter<K, V> {
+
+		public BroadcastStateMetaInfoWriterV4(
+				final RegisteredBroadcastBackendStateMetaInfo.Snapshot<K, V> broadcastStateMetaInfo) {
+			super(broadcastStateMetaInfo);
+		}
+
+		@Override
+		public void writeBroadcastStateMetaInfo(final DataOutputView out) throws IOException {
+			out.writeUTF(broadcastStateMetaInfo.getName());
+			out.writeByte(broadcastStateMetaInfo.getAssignmentMode().ordinal());
+
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(
+				out,
+				broadcastStateMetaInfo.getKeySerializerConfigSnapshot());
+
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(
+				out,
+				broadcastStateMetaInfo.getValueSerializerConfigSnapshot());
+		}
+	}
+
 	// -------------------------------------------------------------------------------
 	//  Readers
 	//   - v1: Flink 1.2.x
 	//   - v2: Flink 1.3.x
 	//   - v3: Flink 1.5.x
+	//   - v4: Flink 1.6.x
 	// -------------------------------------------------------------------------------
 
 	public static <S> OperatorBackendStateMetaInfoReader<S> getOperatorStateReaderForVersion(
@@ -190,10 +242,13 @@ public void writeBroadcastStateMetaInfo(final DataOutputView out) throws IOExcep
 			case 1:
 				return new OperatorBackendStateMetaInfoReaderV1<>(userCodeClassLoader);
 
-			// version 2 and version 3 (current)
 			case 2:
+			case 3:
+				return new OperatorBackendStateMetaInfoReaderV2V3<>(userCodeClassLoader);
+
+			// current version
 			case OperatorBackendSerializationProxy.VERSION:
-				return new OperatorBackendStateMetaInfoReaderV2<>(userCodeClassLoader);
+				return new OperatorBackendStateMetaInfoReaderV4<>(userCodeClassLoader);
 
 			default:
 				// guard for future
@@ -206,9 +261,12 @@ public void writeBroadcastStateMetaInfo(final DataOutputView out) throws IOExcep
 			int version, ClassLoader userCodeClassLoader) {
 
 		switch (version) {
+			case 3:
+				return new BroadcastStateMetaInfoReaderV3<>(userCodeClassLoader);
+
 			// current version
 			case OperatorBackendSerializationProxy.VERSION:
-				return new BroadcastStateMetaInfoReaderV3<>(userCodeClassLoader);
+				return new BroadcastStateMetaInfoReaderV4<>(userCodeClassLoader);
 
 			default:
 				// guard for future
@@ -262,30 +320,31 @@ public OperatorBackendStateMetaInfoReaderV1(ClassLoader userCodeClassLoader) {
 
 			DataInputViewStream dis = new DataInputViewStream(in);
 			ClassLoader previousClassLoader = Thread.currentThread().getContextClassLoader();
+			TypeSerializer<S> stateSerializer;
 			try (
 				InstantiationUtil.FailureTolerantObjectInputStream ois =
 					new InstantiationUtil.FailureTolerantObjectInputStream(dis, userCodeClassLoader)) {
 
 				Thread.currentThread().setContextClassLoader(userCodeClassLoader);
-				TypeSerializer<S> stateSerializer = (TypeSerializer<S>) ois.readObject();
-				stateMetaInfo.setPartitionStateSerializer(stateSerializer);
+				stateSerializer = (TypeSerializer<S>) ois.readObject();
 			} catch (ClassNotFoundException exception) {
 				throw new IOException(exception);
 			} finally {
 				Thread.currentThread().setContextClassLoader(previousClassLoader);
 			}
 
-			// old versions do not contain the partition state serializer's configuration snapshot
-			stateMetaInfo.setPartitionStateSerializerConfigSnapshot(null);
+			// old versions do not contain the partition state serializer's configuration snapshot;
+			// we deserialize the written serializers, and then simply take a snapshot of them now
+			stateMetaInfo.setPartitionStateSerializerConfigSnapshot(stateSerializer.snapshotConfiguration());
 
 			return stateMetaInfo;
 		}
 	}
 
 	@SuppressWarnings("unchecked")
-	public static class OperatorBackendStateMetaInfoReaderV2<S> extends AbstractOperatorBackendStateMetaInfoReader<S> {
+	public static class OperatorBackendStateMetaInfoReaderV2V3<S> extends AbstractOperatorBackendStateMetaInfoReader<S> {
 
-		public OperatorBackendStateMetaInfoReaderV2(ClassLoader userCodeClassLoader) {
+		public OperatorBackendStateMetaInfoReaderV2V3(ClassLoader userCodeClassLoader) {
 			super(userCodeClassLoader);
 		}
 
@@ -300,13 +359,35 @@ public OperatorBackendStateMetaInfoReaderV2(ClassLoader userCodeClassLoader) {
 			Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> stateSerializerAndConfig =
 				TypeSerializerSerializationUtil.readSerializersAndConfigsWithResilience(in, userCodeClassLoader).get(0);
 
-			stateMetaInfo.setPartitionStateSerializer((TypeSerializer<S>) stateSerializerAndConfig.f0);
-			stateMetaInfo.setPartitionStateSerializerConfigSnapshot(stateSerializerAndConfig.f1);
+			stateMetaInfo.setPartitionStateSerializerConfigSnapshot(
+				new BackwardsCompatibleConfigSnapshot(stateSerializerAndConfig.f1, stateSerializerAndConfig.f0));
+
+			return stateMetaInfo;
+		}
+	}
+
+	public static class OperatorBackendStateMetaInfoReaderV4<S> extends AbstractOperatorBackendStateMetaInfoReader<S> {
+
+		public OperatorBackendStateMetaInfoReaderV4(ClassLoader userCodeClassLoader) {
+			super(userCodeClassLoader);
+		}
+
+		@Override
+		public RegisteredOperatorBackendStateMetaInfo.Snapshot<S> readOperatorStateMetaInfo(DataInputView in) throws IOException {
+			RegisteredOperatorBackendStateMetaInfo.Snapshot<S> stateMetaInfo =
+				new RegisteredOperatorBackendStateMetaInfo.Snapshot<>();
+
+			stateMetaInfo.setName(in.readUTF());
+			stateMetaInfo.setAssignmentMode(OperatorStateHandle.Mode.values()[in.readByte()]);
+
+			stateMetaInfo.setPartitionStateSerializerConfigSnapshot(
+				TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(in, userCodeClassLoader));
 
 			return stateMetaInfo;
 		}
 	}
 
+	@SuppressWarnings("unchecked")
 	public static class BroadcastStateMetaInfoReaderV3<K, V> extends AbstractBroadcastStateMetaInfoReader<K, V> {
 
 		public BroadcastStateMetaInfoReaderV3(final ClassLoader userCodeClassLoader) {
@@ -327,11 +408,35 @@ public BroadcastStateMetaInfoReaderV3(final ClassLoader userCodeClassLoader) {
 			Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> keySerializerAndConfig = serializers.get(0);
 			Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> valueSerializerAndConfig = serializers.get(1);
 
-			stateMetaInfo.setKeySerializer((TypeSerializer<K>) keySerializerAndConfig.f0);
-			stateMetaInfo.setKeySerializerConfigSnapshot(keySerializerAndConfig.f1);
-			
-			stateMetaInfo.setValueSerializer((TypeSerializer<V>) valueSerializerAndConfig.f0);
-			stateMetaInfo.setValueSerializerConfigSnapshot(valueSerializerAndConfig.f1);
+			stateMetaInfo.setKeySerializerConfigSnapshot(
+				new BackwardsCompatibleConfigSnapshot(keySerializerAndConfig.f1, keySerializerAndConfig.f0));
+
+			stateMetaInfo.setValueSerializerConfigSnapshot(
+				new BackwardsCompatibleConfigSnapshot(valueSerializerAndConfig.f1, valueSerializerAndConfig.f0));
+
+			return stateMetaInfo;
+		}
+	}
+
+	public static class BroadcastStateMetaInfoReaderV4<K, V> extends AbstractBroadcastStateMetaInfoReader<K, V> {
+
+		public BroadcastStateMetaInfoReaderV4(final ClassLoader userCodeClassLoader) {
+			super(userCodeClassLoader);
+		}
+
+		@Override
+		public RegisteredBroadcastBackendStateMetaInfo.Snapshot<K, V> readBroadcastStateMetaInfo(DataInputView in) throws IOException {
+			RegisteredBroadcastBackendStateMetaInfo.Snapshot<K, V> stateMetaInfo =
+					new RegisteredBroadcastBackendStateMetaInfo.Snapshot<>();
+
+			stateMetaInfo.setName(in.readUTF());
+			stateMetaInfo.setAssignmentMode(OperatorStateHandle.Mode.values()[in.readByte()]);
+
+			stateMetaInfo.setKeySerializerConfigSnapshot(
+				TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(in, userCodeClassLoader));
+
+			stateMetaInfo.setValueSerializerConfigSnapshot(
+				TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(in, userCodeClassLoader));
 
 			return stateMetaInfo;
 		}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBroadcastBackendStateMetaInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBroadcastBackendStateMetaInfo.java
index 7204cd38563..a491559840c 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBroadcastBackendStateMetaInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBroadcastBackendStateMetaInfo.java
@@ -140,8 +140,6 @@ public String toString() {
 
 		private String name;
 		private OperatorStateHandle.Mode assignmentMode;
-		private TypeSerializer<K> keySerializer;
-		private TypeSerializer<V> valueSerializer;
 		private TypeSerializerConfigSnapshot keySerializerConfigSnapshot;
 		private TypeSerializerConfigSnapshot valueSerializerConfigSnapshot;
 
@@ -158,8 +156,8 @@ private Snapshot(
 
 			this.name = Preconditions.checkNotNull(name);
 			this.assignmentMode = Preconditions.checkNotNull(assignmentMode);
-			this.keySerializer = Preconditions.checkNotNull(keySerializer);
-			this.valueSerializer = Preconditions.checkNotNull(valueSerializer);
+			Preconditions.checkNotNull(keySerializer);
+			Preconditions.checkNotNull(valueSerializer);
 			this.keySerializerConfigSnapshot = Preconditions.checkNotNull(keySerializerConfigSnapshot);
 			this.valueSerializerConfigSnapshot = Preconditions.checkNotNull(valueSerializerConfigSnapshot);
 		}
@@ -180,22 +178,6 @@ void setAssignmentMode(OperatorStateHandle.Mode mode) {
 			this.assignmentMode = mode;
 		}
 
-		public TypeSerializer<K> getKeySerializer() {
-			return keySerializer;
-		}
-
-		void setKeySerializer(TypeSerializer<K> serializer) {
-			this.keySerializer = serializer;
-		}
-
-		public TypeSerializer<V> getValueSerializer() {
-			return valueSerializer;
-		}
-
-		void setValueSerializer(TypeSerializer<V> serializer) {
-			this.valueSerializer = serializer;
-		}
-
 		public TypeSerializerConfigSnapshot getKeySerializerConfigSnapshot() {
 			return keySerializerConfigSnapshot;
 		}
@@ -227,8 +209,6 @@ public boolean equals(Object obj) {
 
 			return name.equals(snapshot.getName())
 					&& assignmentMode.ordinal() == snapshot.getAssignmentMode().ordinal()
-					&& Objects.equals(keySerializer, snapshot.getKeySerializer())
-					&& Objects.equals(valueSerializer, snapshot.getValueSerializer())
 					&& keySerializerConfigSnapshot.equals(snapshot.getKeySerializerConfigSnapshot())
 					&& valueSerializerConfigSnapshot.equals(snapshot.getValueSerializerConfigSnapshot());
 		}
@@ -237,8 +217,6 @@ public boolean equals(Object obj) {
 		public int hashCode() {
 			int result = name.hashCode();
 			result = 31 * result + assignmentMode.hashCode();
-			result = 31 * result + ((keySerializer != null) ? keySerializer.hashCode() : 0);
-			result = 31 * result + ((valueSerializer != null) ? valueSerializer.hashCode() : 0);
 			result = 31 * result + keySerializerConfigSnapshot.hashCode();
 			result = 31 * result + valueSerializerConfigSnapshot.hashCode();
 			return result;
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyedBackendStateMetaInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyedBackendStateMetaInfo.java
index c7952ef985c..4f43439b0a0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyedBackendStateMetaInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyedBackendStateMetaInfo.java
@@ -19,13 +19,9 @@
 package org.apache.flink.runtime.state;
 
 import org.apache.flink.api.common.state.StateDescriptor;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.util.Preconditions;
-import org.apache.flink.util.StateMigrationException;
 
 import java.util.Objects;
 
@@ -78,9 +74,7 @@ public String getName() {
 			stateType,
 			name,
 			namespaceSerializer.duplicate(),
-			stateSerializer.duplicate(),
-			namespaceSerializer.snapshotConfiguration(),
-			stateSerializer.snapshotConfiguration());
+			stateSerializer.duplicate());
 	}
 
 	@Override
@@ -133,10 +127,8 @@ public int hashCode() {
 
 		private StateDescriptor.Type stateType;
 		private String name;
-		private TypeSerializer<N> namespaceSerializer;
-		private TypeSerializer<S> stateSerializer;
-		private TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot;
-		private TypeSerializerConfigSnapshot stateSerializerConfigSnapshot;
+		private TypeSerializerConfigSnapshot<N> namespaceSerializerConfigSnapshot;
+		private TypeSerializerConfigSnapshot<S> stateSerializerConfigSnapshot;
 
 		/** Empty constructor used when restoring the state meta info snapshot. */
 		Snapshot() {}
@@ -145,16 +137,15 @@ private Snapshot(
 				StateDescriptor.Type stateType,
 				String name,
 				TypeSerializer<N> namespaceSerializer,
-				TypeSerializer<S> stateSerializer,
-				TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot,
-				TypeSerializerConfigSnapshot stateSerializerConfigSnapshot) {
+				TypeSerializer<S> stateSerializer) {
 
 			this.stateType = Preconditions.checkNotNull(stateType);
 			this.name = Preconditions.checkNotNull(name);
-			this.namespaceSerializer = Preconditions.checkNotNull(namespaceSerializer);
-			this.stateSerializer = Preconditions.checkNotNull(stateSerializer);
-			this.namespaceSerializerConfigSnapshot = Preconditions.checkNotNull(namespaceSerializerConfigSnapshot);
-			this.stateSerializerConfigSnapshot = Preconditions.checkNotNull(stateSerializerConfigSnapshot);
+
+			Preconditions.checkNotNull(namespaceSerializer);
+			Preconditions.checkNotNull(stateSerializer);
+			this.namespaceSerializerConfigSnapshot = Preconditions.checkNotNull(namespaceSerializer.snapshotConfiguration());
+			this.stateSerializerConfigSnapshot = Preconditions.checkNotNull(stateSerializer.snapshotConfiguration());
 		}
 
 		public StateDescriptor.Type getStateType() {
@@ -173,35 +164,19 @@ void setName(String name) {
 			this.name = name;
 		}
 
-		public TypeSerializer<N> getNamespaceSerializer() {
-			return namespaceSerializer;
-		}
-
-		void setNamespaceSerializer(TypeSerializer<N> namespaceSerializer) {
-			this.namespaceSerializer = namespaceSerializer;
-		}
-
-		public TypeSerializer<S> getStateSerializer() {
-			return stateSerializer;
-		}
-
-		void setStateSerializer(TypeSerializer<S> stateSerializer) {
-			this.stateSerializer = stateSerializer;
-		}
-
-		public TypeSerializerConfigSnapshot getNamespaceSerializerConfigSnapshot() {
+		public TypeSerializerConfigSnapshot<N> getNamespaceSerializerConfigSnapshot() {
 			return namespaceSerializerConfigSnapshot;
 		}
 
-		void setNamespaceSerializerConfigSnapshot(TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot) {
+		void setNamespaceSerializerConfigSnapshot(TypeSerializerConfigSnapshot<N> namespaceSerializerConfigSnapshot) {
 			this.namespaceSerializerConfigSnapshot = namespaceSerializerConfigSnapshot;
 		}
 
-		public TypeSerializerConfigSnapshot getStateSerializerConfigSnapshot() {
+		public TypeSerializerConfigSnapshot<S> getStateSerializerConfigSnapshot() {
 			return stateSerializerConfigSnapshot;
 		}
 
-		void setStateSerializerConfigSnapshot(TypeSerializerConfigSnapshot stateSerializerConfigSnapshot) {
+		void setStateSerializerConfigSnapshot(TypeSerializerConfigSnapshot<S> stateSerializerConfigSnapshot) {
 			this.stateSerializerConfigSnapshot = stateSerializerConfigSnapshot;
 		}
 
@@ -226,9 +201,7 @@ public boolean equals(Object o) {
 			}
 
 			// need to check for nulls because serializer and config snapshots may be null on restore
-			return Objects.equals(getStateSerializer(), that.getStateSerializer())
-				&& Objects.equals(getNamespaceSerializer(), that.getNamespaceSerializer())
-				&& Objects.equals(getNamespaceSerializerConfigSnapshot(), that.getNamespaceSerializerConfigSnapshot())
+			return Objects.equals(getNamespaceSerializerConfigSnapshot(), that.getNamespaceSerializerConfigSnapshot())
 				&& Objects.equals(getStateSerializerConfigSnapshot(), that.getStateSerializerConfigSnapshot());
 		}
 
@@ -237,63 +210,9 @@ public int hashCode() {
 			// need to check for nulls because serializer and config snapshots may be null on restore
 			int result = getName().hashCode();
 			result = 31 * result + getStateType().hashCode();
-			result = 31 * result + (getNamespaceSerializer() != null ? getNamespaceSerializer().hashCode() : 0);
-			result = 31 * result + (getStateSerializer() != null ? getStateSerializer().hashCode() : 0);
 			result = 31 * result + (getNamespaceSerializerConfigSnapshot() != null ? getNamespaceSerializerConfigSnapshot().hashCode() : 0);
 			result = 31 * result + (getStateSerializerConfigSnapshot() != null ? getStateSerializerConfigSnapshot().hashCode() : 0);
 			return result;
 		}
 	}
-
-	/**
-	 * Checks compatibility of a restored k/v state, with the new {@link StateDescriptor} provided to it.
-	 * This checks that the descriptor specifies identical names and state types, as well as
-	 * serializers that are compatible for the restored k/v state bytes.
-	 */
-	public static  <N, S> RegisteredKeyedBackendStateMetaInfo<N, S> resolveKvStateCompatibility(
-		RegisteredKeyedBackendStateMetaInfo.Snapshot<N, S> restoredStateMetaInfoSnapshot,
-		TypeSerializer<N> newNamespaceSerializer,
-		StateDescriptor<?, S> newStateDescriptor) throws StateMigrationException {
-
-		Preconditions.checkState(
-			Objects.equals(newStateDescriptor.getName(), restoredStateMetaInfoSnapshot.getName()),
-			"Incompatible state names. " +
-				"Was [" + restoredStateMetaInfoSnapshot.getName() + "], " +
-				"registered with [" + newStateDescriptor.getName() + "].");
-
-		if (!Objects.equals(newStateDescriptor.getType(), StateDescriptor.Type.UNKNOWN)
-			&& !Objects.equals(restoredStateMetaInfoSnapshot.getStateType(), StateDescriptor.Type.UNKNOWN)) {
-
-			Preconditions.checkState(
-				newStateDescriptor.getType() == restoredStateMetaInfoSnapshot.getStateType(),
-				"Incompatible state types. " +
-					"Was [" + restoredStateMetaInfoSnapshot.getStateType() + "], " +
-					"registered with [" + newStateDescriptor.getType() + "].");
-		}
-
-		// check compatibility results to determine if state migration is required
-		CompatibilityResult<N> namespaceCompatibility = CompatibilityUtil.resolveCompatibilityResult(
-			restoredStateMetaInfoSnapshot.getNamespaceSerializer(),
-			null,
-			restoredStateMetaInfoSnapshot.getNamespaceSerializerConfigSnapshot(),
-			newNamespaceSerializer);
-
-		TypeSerializer<S> newStateSerializer = newStateDescriptor.getSerializer();
-		CompatibilityResult<S> stateCompatibility = CompatibilityUtil.resolveCompatibilityResult(
-			restoredStateMetaInfoSnapshot.getStateSerializer(),
-			UnloadableDummyTypeSerializer.class,
-			restoredStateMetaInfoSnapshot.getStateSerializerConfigSnapshot(),
-			newStateSerializer);
-
-		if (namespaceCompatibility.isRequiresMigration() || stateCompatibility.isRequiresMigration()) {
-			// TODO state migration currently isn't possible.
-			throw new StateMigrationException("State migration isn't supported, yet.");
-		} else {
-			return new RegisteredKeyedBackendStateMetaInfo<>(
-				newStateDescriptor.getType(),
-				newStateDescriptor.getName(),
-				newNamespaceSerializer,
-				newStateSerializer);
-		}
-	}
 }
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorBackendStateMetaInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorBackendStateMetaInfo.java
index a9adc8d1e3a..9d8428f8a3b 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorBackendStateMetaInfo.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorBackendStateMetaInfo.java
@@ -133,7 +133,6 @@ public String toString() {
 
 		private String name;
 		private OperatorStateHandle.Mode assignmentMode;
-		private TypeSerializer<S> partitionStateSerializer;
 		private TypeSerializerConfigSnapshot partitionStateSerializerConfigSnapshot;
 
 		/** Empty constructor used when restoring the state meta info snapshot. */
@@ -147,7 +146,7 @@ private Snapshot(
 
 			this.name = Preconditions.checkNotNull(name);
 			this.assignmentMode = Preconditions.checkNotNull(assignmentMode);
-			this.partitionStateSerializer = Preconditions.checkNotNull(partitionStateSerializer);
+			Preconditions.checkNotNull(partitionStateSerializer);
 			this.partitionStateSerializerConfigSnapshot = Preconditions.checkNotNull(partitionStateSerializerConfigSnapshot);
 		}
 
@@ -167,14 +166,6 @@ void setAssignmentMode(OperatorStateHandle.Mode assignmentMode) {
 			this.assignmentMode = assignmentMode;
 		}
 
-		public TypeSerializer<S> getPartitionStateSerializer() {
-			return partitionStateSerializer;
-		}
-
-		void setPartitionStateSerializer(TypeSerializer<S> partitionStateSerializer) {
-			this.partitionStateSerializer = partitionStateSerializer;
-		}
-
 		public TypeSerializerConfigSnapshot getPartitionStateSerializerConfigSnapshot() {
 			return partitionStateSerializerConfigSnapshot;
 		}
@@ -202,7 +193,6 @@ public boolean equals(Object obj) {
 			// need to check for nulls because serializer and config snapshots may be null on restore
 			return name.equals(snapshot.getName())
 				&& assignmentMode.equals(snapshot.getAssignmentMode())
-				&& Objects.equals(partitionStateSerializer, snapshot.getPartitionStateSerializer())
 				&& Objects.equals(partitionStateSerializerConfigSnapshot, snapshot.getPartitionStateSerializerConfigSnapshot());
 		}
 
@@ -211,7 +201,6 @@ public int hashCode() {
 			// need to check for nulls because serializer and config snapshots may be null on restore
 			int result = getName().hashCode();
 			result = 31 * result + getAssignmentMode().hashCode();
-			result = 31 * result + (getPartitionStateSerializer() != null ? getPartitionStateSerializer().hashCode() : 0);
 			result = 31 * result + (getPartitionStateSerializerConfigSnapshot() != null ? getPartitionStateSerializerConfigSnapshot().hashCode() : 0);
 			return result;
 		}
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtil.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtil.java
index f129c318110..1da1d532652 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtil.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/StateUtil.java
@@ -18,12 +18,15 @@
 
 package org.apache.flink.runtime.state;
 
+import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.util.FutureUtil;
 import org.apache.flink.util.LambdaUtil;
+import org.apache.flink.util.Preconditions;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.util.Objects;
 import java.util.concurrent.CancellationException;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.RunnableFuture;
@@ -85,4 +88,24 @@ public static void discardStateFuture(RunnableFuture<? extends StateObject> stat
 			}
 		}
 	}
+
+	public static <N, S> void checkStateTypeCompatibility(
+		RegisteredKeyedBackendStateMetaInfo.Snapshot<N, S> restoredStateMetaInfoSnapshot,
+		StateDescriptor<?, S> newStateDescriptor) {
+		Preconditions.checkState(
+			Objects.equals(newStateDescriptor.getName(), restoredStateMetaInfoSnapshot.getName()),
+			"Incompatible state names. " +
+				"Was [" + restoredStateMetaInfoSnapshot.getName() + "], " +
+				"registered with [" + newStateDescriptor.getName() + "].");
+
+		if (!Objects.equals(newStateDescriptor.getType(), StateDescriptor.Type.UNKNOWN)
+			&& !Objects.equals(restoredStateMetaInfoSnapshot.getStateType(), StateDescriptor.Type.UNKNOWN)) {
+
+			Preconditions.checkState(
+				newStateDescriptor.getType() == restoredStateMetaInfoSnapshot.getStateType(),
+				"Incompatible state types. " +
+					"Was [" + restoredStateMetaInfoSnapshot.getStateType() + "], " +
+					"registered with [" + newStateDescriptor.getType() + "].");
+		}
+	}
 }
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
index 82ce5847627..0e1238e48e0 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java
@@ -30,7 +30,6 @@
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.CompatibilityUtil;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.fs.FSDataInputStream;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
@@ -56,6 +55,7 @@
 import org.apache.flink.runtime.state.SnapshotResult;
 import org.apache.flink.runtime.state.SnapshotStrategy;
 import org.apache.flink.runtime.state.StateSnapshot;
+import org.apache.flink.runtime.state.StateUtil;
 import org.apache.flink.runtime.state.StreamCompressionDecorator;
 import org.apache.flink.runtime.state.StreamStateHandle;
 import org.apache.flink.runtime.state.UncompressedStreamCompressionDecorator;
@@ -180,10 +180,13 @@ public HeapKeyedStateBackend(
 				"Requested to check compatibility of a restored RegisteredKeyedBackendStateMetaInfo," +
 					" but its corresponding restored snapshot cannot be found.");
 
-			newMetaInfo = RegisteredKeyedBackendStateMetaInfo.resolveKvStateCompatibility(
-				restoredMetaInfoSnapshot,
+			StateUtil.checkStateTypeCompatibility(restoredMetaInfoSnapshot, stateDesc);
+
+			newMetaInfo = new RegisteredKeyedBackendStateMetaInfo<>(
+				stateDesc.getType(),
+				stateDesc.getName(),
 				namespaceSerializer,
-				stateDesc);
+				stateDesc.getSerializer());
 
 			stateTable.setMetaInfo(newMetaInfo);
 		} else {
@@ -292,11 +295,9 @@ private void restorePartitionedState(Collection<KeyedStateHandle> state) throws
 					// check for key serializer compatibility; this also reconfigures the
 					// key serializer to be compatible, if it is required and is possible
 					if (CompatibilityUtil.resolveCompatibilityResult(
-							serializationProxy.getKeySerializer(),
-							UnloadableDummyTypeSerializer.class,
 							serializationProxy.getKeySerializerConfigSnapshot(),
 							keySerializer)
-						.isRequiresMigration()) {
+						.isIncompatible()) {
 
 						// TODO replace with state migration; note that key hash codes need to remain the same after migration
 						throw new StateMigrationException("The new key serializer is not compatible to read previous keys. " +
@@ -321,8 +322,8 @@ private void restorePartitionedState(Collection<KeyedStateHandle> state) throws
 								new RegisteredKeyedBackendStateMetaInfo<>(
 									restoredMetaInfo.getStateType(),
 									restoredMetaInfo.getName(),
-									restoredMetaInfo.getNamespaceSerializer(),
-									restoredMetaInfo.getStateSerializer());
+									restoredMetaInfo.getNamespaceSerializerConfigSnapshot().restoreSerializer(),
+									restoredMetaInfo.getStateSerializerConfigSnapshot().restoreSerializer());
 
 						stateTable = snapshotStrategy.newStateTable(registeredKeyedBackendStateMetaInfo);
 						stateTables.put(restoredMetaInfo.getName(), stateTable);
diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTableByKeyGroupReaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTableByKeyGroupReaders.java
index 0b69a875a06..e08e90eb44f 100644
--- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTableByKeyGroupReaders.java
+++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTableByKeyGroupReaders.java
@@ -49,6 +49,7 @@
 			case 2:
 			case 3:
 			case 4:
+			case 5:
 				return new StateTableByKeyGroupReaderV2V3<>(table);
 			default:
 				throw new IllegalArgumentException("Unknown version: " + version);
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListSerializer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListSerializer.java
index 4bdc5e85cdd..ee9e3d202e1 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListSerializer.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListSerializer.java
@@ -21,7 +21,7 @@
 import java.io.IOException;
 import java.util.Arrays;
 
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.core.memory.DataInputView;
@@ -131,12 +131,12 @@ public int hashCode() {
 	}
 
 	@Override
-	public TypeSerializerConfigSnapshot snapshotConfiguration() {
+	public TypeSerializerConfigSnapshot<IntList> snapshotConfiguration() {
 		throw new UnsupportedOperationException();
 	}
 
 	@Override
-	public CompatibilityResult<IntList> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	public TypeSerializerSchemaCompatibility<IntList> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		throw new UnsupportedOperationException();
 	}
 }
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairSerializer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairSerializer.java
index 0ae5e71d428..39c39da34f9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairSerializer.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairSerializer.java
@@ -21,7 +21,7 @@
 
 import java.io.IOException;
 
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializerFactory;
@@ -140,12 +140,12 @@ public boolean equals(Object obj) {
 	}
 
 	@Override
-	public TypeSerializerConfigSnapshot snapshotConfiguration() {
+	public TypeSerializerConfigSnapshot<IntPair> snapshotConfiguration() {
 		throw new UnsupportedOperationException();
 	}
 
 	@Override
-	public CompatibilityResult<IntPair> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	public TypeSerializerSchemaCompatibility<IntPair> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		throw new UnsupportedOperationException();
 	}
 }
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairSerializer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairSerializer.java
index 17ee5f1530b..6d3bd7aa8f9 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairSerializer.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairSerializer.java
@@ -20,7 +20,7 @@
 
 import java.io.IOException;
 
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.core.memory.DataInputView;
@@ -108,12 +108,12 @@ public int hashCode() {
 	}
 
 	@Override
-	public TypeSerializerConfigSnapshot snapshotConfiguration() {
+	public TypeSerializerConfigSnapshot<StringPair> snapshotConfiguration() {
 		throw new UnsupportedOperationException();
 	}
 
 	@Override
-	public CompatibilityResult<StringPair> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	public TypeSerializerSchemaCompatibility<StringPair> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		throw new UnsupportedOperationException();
 	}
 }
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateRegistryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateRegistryTest.java
index c1c56bf250b..2db807cfdfb 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateRegistryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/query/KvStateRegistryTest.java
@@ -19,9 +19,9 @@
 package org.apache.flink.runtime.query;
 
 import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -401,12 +401,12 @@ public int hashCode() {
 		}
 
 		@Override
-		public TypeSerializerConfigSnapshot snapshotConfiguration() {
+		public TypeSerializerConfigSnapshot<String> snapshotConfiguration() {
 			return null;
 		}
 
 		@Override
-		public CompatibilityResult<String> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+		public TypeSerializerSchemaCompatibility<String> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 			return null;
 		}
 	}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendMigrationTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendMigrationTest.java
new file mode 100644
index 00000000000..a2c195ea7dd
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/FileStateBackendMigrationTest.java
@@ -0,0 +1,61 @@
+/*
+ * 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.runtime.state;
+
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for the keyed state backend and operator state backend, as created by the
+ * {@link FsStateBackend}.
+ */
+public class FileStateBackendMigrationTest extends StateBackendMigrationTestBase<FsStateBackend> {
+
+	@Rule
+	public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+	@Override
+	protected FsStateBackend getStateBackend() throws Exception {
+		File checkpointPath = tempFolder.newFolder();
+		return new FsStateBackend(checkpointPath.toURI(), false);
+	}
+
+	@Override
+	protected void verifyValueStateWithIncompatibleSerializerCounts() {
+		assertEquals((Integer) 2, CustomStringSerializer.serializeCalled.get(SerializerVersion.INITIAL));
+		assertEquals((Integer) 2, CustomStringSerializer.serializeCalled.get(SerializerVersion.NEW));
+		assertEquals((Integer) 2, CustomStringSerializer.deserializeCalled.get(SerializerVersion.NEW));
+		assertEquals((Integer) 2, CustomStringSerializer.deserializeCalled.get(SerializerVersion.RESTORE));
+	}
+
+	@Override
+	protected void verifyValueStateWithReconfiguredSerializerCounts() {
+		assertEquals((Integer) 2, CustomStringSerializer.serializeCalled.get(SerializerVersion.INITIAL));
+		assertEquals((Integer) 2, CustomStringSerializer.serializeCalled.get(SerializerVersion.NEW));
+		assertEquals((Integer) 2, CustomStringSerializer.deserializeCalled.get(SerializerVersion.NEW));
+		assertEquals((Integer) 2, CustomStringSerializer.deserializeCalled.get(SerializerVersion.RESTORE));
+	}
+
+}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStateBackendTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStateBackendTest.java
index d8918e78478..c6f9c4294d7 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStateBackendTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/OperatorStateBackendTest.java
@@ -23,7 +23,7 @@
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.state.MapStateDescriptor;
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
@@ -351,13 +351,14 @@ public int hashCode() {
 		}
 
 		@Override
-		public TypeSerializerConfigSnapshot snapshotConfiguration() {
+		public TypeSerializerConfigSnapshot<Integer> snapshotConfiguration() {
 			return IntSerializer.INSTANCE.snapshotConfiguration();
 		}
 
 		@Override
-		public CompatibilityResult<Integer> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-			return IntSerializer.INSTANCE.ensureCompatibility(configSnapshot);
+		public TypeSerializerSchemaCompatibility<Integer> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
+			// TODO this method will be removed in the follow-up commits
+			return null;
 		}
 	}
 
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/SerializationProxiesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/SerializationProxiesTest.java
index 3f78097f11c..30dad1a3aa4 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/SerializationProxiesTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/SerializationProxiesTest.java
@@ -20,7 +20,6 @@
 
 import org.apache.flink.api.common.state.StateDescriptor;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.api.common.typeutils.base.DoubleSerializer;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
@@ -74,7 +73,6 @@ public void testKeyedBackendSerializationProxyRoundtrip() throws Exception {
 		}
 
 		Assert.assertEquals(true, serializationProxy.isUsingKeyGroupCompression());
-		Assert.assertEquals(keySerializer, serializationProxy.getKeySerializer());
 		Assert.assertEquals(keySerializer.snapshotConfiguration(), serializationProxy.getKeySerializerConfigSnapshot());
 		Assert.assertEquals(stateMetaInfoList, serializationProxy.getStateMetaInfoSnapshots());
 	}
@@ -123,12 +121,9 @@ public void testKeyedBackendSerializationProxyRoundtripWithSerializerSerializati
 		}
 
 		Assert.assertEquals(true, serializationProxy.isUsingKeyGroupCompression());
-		Assert.assertTrue(serializationProxy.getKeySerializer() instanceof UnloadableDummyTypeSerializer);
 		Assert.assertEquals(keySerializer.snapshotConfiguration(), serializationProxy.getKeySerializerConfigSnapshot());
 
 		for (RegisteredKeyedBackendStateMetaInfo.Snapshot<?, ?> meta : serializationProxy.getStateMetaInfoSnapshots()) {
-			Assert.assertTrue(meta.getNamespaceSerializer() instanceof UnloadableDummyTypeSerializer);
-			Assert.assertTrue(meta.getStateSerializer() instanceof UnloadableDummyTypeSerializer);
 			Assert.assertEquals(namespaceSerializer.snapshotConfiguration(), meta.getNamespaceSerializerConfigSnapshot());
 			Assert.assertEquals(stateSerializer.snapshotConfiguration(), meta.getStateSerializerConfigSnapshot());
 		}
@@ -195,8 +190,6 @@ public void testKeyedStateMetaInfoReadSerializerFailureResilience() throws Excep
 		}
 
 		Assert.assertEquals(name, metaInfo.getName());
-		Assert.assertTrue(metaInfo.getNamespaceSerializer() instanceof UnloadableDummyTypeSerializer);
-		Assert.assertTrue(metaInfo.getStateSerializer() instanceof UnloadableDummyTypeSerializer);
 		Assert.assertEquals(namespaceSerializer.snapshotConfiguration(), metaInfo.getNamespaceSerializerConfigSnapshot());
 		Assert.assertEquals(stateSerializer.snapshotConfiguration(), metaInfo.getStateSerializerConfigSnapshot());
 	}
@@ -271,7 +264,6 @@ public void testOperatorStateMetaInfoSerialization() throws Exception {
 
 		Assert.assertEquals(name, metaInfo.getName());
 		Assert.assertEquals(OperatorStateHandle.Mode.UNION, metaInfo.getAssignmentMode());
-		Assert.assertEquals(stateSerializer, metaInfo.getPartitionStateSerializer());
 	}
 
 	@Test
@@ -302,8 +294,6 @@ public void testBroadcastStateMetaInfoSerialization() throws Exception {
 
 		Assert.assertEquals(name, metaInfo.getName());
 		Assert.assertEquals(OperatorStateHandle.Mode.BROADCAST, metaInfo.getAssignmentMode());
-		Assert.assertEquals(keySerializer, metaInfo.getKeySerializer());
-		Assert.assertEquals(valueSerializer, metaInfo.getValueSerializer());
 	}
 
 	@Test
@@ -339,7 +329,6 @@ public void testOperatorStateMetaInfoReadSerializerFailureResilience() throws Ex
 		}
 
 		Assert.assertEquals(name, metaInfo.getName());
-		Assert.assertTrue(metaInfo.getPartitionStateSerializer() instanceof UnloadableDummyTypeSerializer);
 		Assert.assertEquals(stateSerializer.snapshotConfiguration(), metaInfo.getPartitionStateSerializerConfigSnapshot());
 	}
 
@@ -377,9 +366,7 @@ public void testBroadcastStateMetaInfoReadSerializerFailureResilience() throws E
 		}
 
 		Assert.assertEquals(broadcastName, broadcastMetaInfo.getName());
-		Assert.assertTrue(broadcastMetaInfo.getKeySerializer() instanceof UnloadableDummyTypeSerializer);
 		Assert.assertEquals(keySerializer.snapshotConfiguration(), broadcastMetaInfo.getKeySerializerConfigSnapshot());
-		Assert.assertTrue(broadcastMetaInfo.getValueSerializer() instanceof UnloadableDummyTypeSerializer);
 		Assert.assertEquals(valueSerializer.snapshotConfiguration(), broadcastMetaInfo.getValueSerializerConfigSnapshot());
 	}
 
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java
new file mode 100644
index 00000000000..e37a7d5aa10
--- /dev/null
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendMigrationTestBase.java
@@ -0,0 +1,637 @@
+/*
+ * 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.runtime.state;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.api.common.state.ValueStateDescriptor;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.checkpoint.StateObjectCollection;
+import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.operators.testutils.DummyEnvironment;
+import org.apache.flink.types.StringValue;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.RunnableFuture;
+
+import static org.junit.Assert.assertEquals;
+
+// TODO: remove print statements
+
+/**
+ * Tests for the {@link KeyedStateBackend} and {@link OperatorStateBackend} as produced
+ * by various {@link StateBackend}s.
+ */
+@SuppressWarnings("serial")
+public abstract class StateBackendMigrationTestBase<B extends AbstractStateBackend> extends TestLogger {
+	@Rule
+	public final ExpectedException expectedException = ExpectedException.none();
+
+	// lazily initialized stream storage
+	private CheckpointStorageLocation checkpointStorageLocation;
+
+	protected abstract B getStateBackend() throws Exception;
+
+	protected CheckpointStreamFactory createStreamFactory() throws Exception {
+		if (checkpointStorageLocation == null) {
+			checkpointStorageLocation = getStateBackend()
+				.createCheckpointStorage(new JobID())
+				.initializeLocationForCheckpoint(1L);
+		}
+
+		return checkpointStorageLocation;
+	}
+
+	protected <K> AbstractKeyedStateBackend<K> createKeyedBackend(TypeSerializer<K> keySerializer) throws Exception {
+		return createKeyedBackend(keySerializer, new DummyEnvironment());
+	}
+
+	protected <K> AbstractKeyedStateBackend<K> createKeyedBackend(TypeSerializer<K> keySerializer, Environment env) throws Exception {
+		return createKeyedBackend(
+			keySerializer,
+			10,
+			new KeyGroupRange(0, 9),
+			env);
+	}
+
+	protected <K> AbstractKeyedStateBackend<K> createKeyedBackend(
+		TypeSerializer<K> keySerializer,
+		int numberOfKeyGroups,
+		KeyGroupRange keyGroupRange,
+		Environment env) throws Exception {
+
+		AbstractKeyedStateBackend<K> backend = getStateBackend().createKeyedStateBackend(
+			env,
+			new JobID(),
+			"test_op",
+			keySerializer,
+			numberOfKeyGroups,
+			keyGroupRange,
+			env.getTaskKvStateRegistry());
+
+		backend.restore(null);
+
+		return backend;
+	}
+
+	protected <K> AbstractKeyedStateBackend<K> restoreKeyedBackend(TypeSerializer<K> keySerializer, KeyedStateHandle state) throws Exception {
+		return restoreKeyedBackend(keySerializer, state, new DummyEnvironment());
+	}
+
+	protected <K> AbstractKeyedStateBackend<K> restoreKeyedBackend(
+		TypeSerializer<K> keySerializer,
+		KeyedStateHandle state,
+		Environment env) throws Exception {
+		return restoreKeyedBackend(
+			keySerializer,
+			10,
+			new KeyGroupRange(0, 9),
+			Collections.singletonList(state),
+			env);
+	}
+
+	protected <K> AbstractKeyedStateBackend<K> restoreKeyedBackend(
+		TypeSerializer<K> keySerializer,
+		int numberOfKeyGroups,
+		KeyGroupRange keyGroupRange,
+		List<KeyedStateHandle> state,
+		Environment env) throws Exception {
+
+		AbstractKeyedStateBackend<K> backend = getStateBackend().createKeyedStateBackend(
+			env,
+			new JobID(),
+			"test_op",
+			keySerializer,
+			numberOfKeyGroups,
+			keyGroupRange,
+			env.getTaskKvStateRegistry());
+
+		backend.restore(new StateObjectCollection<>(state));
+
+		return backend;
+	}
+
+	protected KeyedStateHandle runSnapshot(
+		RunnableFuture<SnapshotResult<KeyedStateHandle>> snapshotRunnableFuture) throws Exception {
+
+		if (!snapshotRunnableFuture.isDone()) {
+			snapshotRunnableFuture.run();
+		}
+
+		SnapshotResult<KeyedStateHandle> snapshotResult = snapshotRunnableFuture.get();
+		return snapshotResult.getJobManagerOwnedSnapshot();
+	}
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testValueStateWithIncompatibleSerializer() throws Exception {
+		CustomStringSerializer.resetCountingMaps();
+		CheckpointStreamFactory streamFactory = createStreamFactory();
+		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
+
+		ValueStateDescriptor<String> kvId = new ValueStateDescriptor<>(
+			"id",
+			new CustomStringSerializer(
+				SerializerCompatibilityType.INCOMPATIBLE, SerializerVersion.INITIAL));
+
+		ValueState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, kvId);
+
+		// some modifications to the state
+		backend.setCurrentKey(1);
+		state.update("1");
+		backend.setCurrentKey(2);
+		state.update("2");
+		backend.setCurrentKey(1);
+
+		System.out.println("SNAPSHOT");
+
+		// draw a snapshot
+		KeyedStateHandle snapshot1 = runSnapshot(backend.snapshot(1L, 2L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()));
+
+		backend.dispose();
+
+		System.out.println("RESTORE");
+
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot1);
+
+		ValueStateDescriptor<String> newKvId = new ValueStateDescriptor<>("id",
+			new CustomStringSerializer(
+				SerializerCompatibilityType.COMPATIBLE_AS_IS, SerializerVersion.NEW));
+
+		ValueState<String> restored1 = backend.getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, newKvId);
+
+		System.out.println("MESSING WITH STATE");
+
+		// some modifications to the state
+		backend.setCurrentKey(1);
+		assertEquals("1", restored1.value());
+		restored1.update("1");
+		backend.setCurrentKey(2);
+		assertEquals("2", restored1.value());
+		restored1.update("3");
+		assertEquals("3", restored1.value());
+
+		// draw another snapshot so that we see serialization from the NEW serialize on
+		// the file backend
+		KeyedStateHandle snapshot2 = runSnapshot(backend.snapshot(2L, 3L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()));
+
+		backend.dispose();
+
+		// and restore once with NEW from NEW so that we see a read using the NEW serializer
+		// on the file backend
+		ValueStateDescriptor<String> newKvId2 = new ValueStateDescriptor<>("id",
+			new CustomStringSerializer(
+				SerializerCompatibilityType.COMPATIBLE_AS_IS, SerializerVersion.NEW));
+
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2);
+		backend.getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, newKvId2);
+
+		snapshot2.discardState();
+		snapshot1.discardState();
+		backend.dispose();
+
+		System.out.println(CustomStringSerializer.serializeCalled);
+		System.out.println(CustomStringSerializer.deserializeCalled);
+		verifyValueStateWithIncompatibleSerializerCounts();
+	}
+
+	protected abstract void verifyValueStateWithIncompatibleSerializerCounts();
+
+	@Test
+	@SuppressWarnings("unchecked")
+	public void testValueStateWithReconfiguredSerializer() throws Exception {
+		CustomStringSerializer.resetCountingMaps();
+		CheckpointStreamFactory streamFactory = createStreamFactory();
+		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
+
+		ValueStateDescriptor<String> kvId = new ValueStateDescriptor<>(
+			"id",
+			new CustomStringSerializer(
+				SerializerCompatibilityType.COMPATIBLE_AFTER_RECONFIGURATION, SerializerVersion.INITIAL));
+
+		ValueState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, kvId);
+
+		// some modifications to the state
+		backend.setCurrentKey(1);
+		state.update("1");
+		backend.setCurrentKey(2);
+		state.update("2");
+		backend.setCurrentKey(1);
+
+		System.out.println("SNAPSHOT");
+
+		// draw a snapshot
+		KeyedStateHandle snapshot1 = runSnapshot(backend.snapshot(1L, 2L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()));
+
+		backend.dispose();
+
+		System.out.println("RESTORE");
+
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot1);
+		snapshot1.discardState();
+
+		ValueStateDescriptor<String> newKvId = new ValueStateDescriptor<>("id",
+			new CustomStringSerializer(
+				SerializerCompatibilityType.COMPATIBLE_AS_IS, SerializerVersion.NEW));
+
+		ValueState<String> restored1 = backend.getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, newKvId);
+
+		System.out.println("MESSING WITH STATE");
+
+		// some modifications to the state
+		backend.setCurrentKey(1);
+		assertEquals("1", restored1.value());
+		restored1.update("1");
+		backend.setCurrentKey(2);
+		assertEquals("2", restored1.value());
+		restored1.update("3");
+		assertEquals("3", restored1.value());
+
+		// draw another snapshot so that we see serialization from the NEW serialize on
+		// the file backend
+		KeyedStateHandle snapshot2 = runSnapshot(backend.snapshot(2L, 3L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()));
+
+		backend.dispose();
+
+		// and restore once with NEW from NEW so that we see a read using the NEW serializer
+		// on the file backend
+		ValueStateDescriptor<String> newKvId2 = new ValueStateDescriptor<>("id",
+			new CustomStringSerializer(
+				SerializerCompatibilityType.COMPATIBLE_AS_IS, SerializerVersion.NEW));
+
+		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot2);
+		backend.getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, newKvId2);
+		snapshot2.discardState();
+		backend.dispose();
+
+		System.out.println(CustomStringSerializer.serializeCalled);
+		System.out.println(CustomStringSerializer.deserializeCalled);
+		verifyValueStateWithReconfiguredSerializerCounts();
+	}
+
+	protected abstract void verifyValueStateWithReconfiguredSerializerCounts();
+
+
+//	@Test
+//	@SuppressWarnings("unchecked")
+//	public void testListState() throws Exception {
+//		CustomStringSerializer.resetCountingMaps();
+//		CheckpointStreamFactory streamFactory = createStreamFactory();
+//		AbstractKeyedStateBackend<Integer> backend = createKeyedBackend(IntSerializer.INSTANCE);
+//
+//		ListStateDescriptor<String> kvId = new ListStateDescriptor<>(
+//			"id",
+//			new CustomStringSerializer(
+//				SerializerCompatibilityType.COMPATIBLE_AFTER_RECONFIGURATION, SerializerVersion.INITIAL));
+//
+//		ListState<String> state = backend.getPartitionedState(VoidNamespace.INSTANCE, CustomVoidNamespaceSerializer.INSTANCE, kvId);
+//
+//		// some modifications to the state
+//		backend.setCurrentKey(1);
+//		assertNull(state.get());
+//		state.add("1");
+//		backend.setCurrentKey(2);
+//		assertNull(state.get());
+//		state.add("2");
+//		backend.setCurrentKey(1);
+//		assertThat(state.get(), containsInAnyOrder("1"));
+//
+//		System.out.println("SNAPSHOT");
+//
+//		// draw a snapshot
+//		KeyedStateHandle snapshot1 = runSnapshot(backend.snapshot(1L, 2L, streamFactory, CheckpointOptions.forCheckpointWithDefaultLocation()));
+//
+//		backend.dispose();
+//
+//		backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot1);
+//		snapshot1.discardState();
+//
+//		ListStateDescriptor<String> newKvId = new ListStateDescriptor<>(
+//			"id",
+//			new CustomStringSerializer(
+//				SerializerCompatibilityType.COMPATIBLE_AFTER_RECONFIGURATION, SerializerVersion.NEW));
+//
+//		System.out.println("RESTORE");
+//		ListState<String> restored1 = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, newKvId);
+//
+//		System.out.println("MESSING WITH STATE");
+//
+//		// some modifications to the state
+//		backend.setCurrentKey(1);
+//		assertThat(state.get(), containsInAnyOrder("1"));
+//		restored1.add("1");
+//		backend.setCurrentKey(2);
+//		assertThat(state.get(), containsInAnyOrder("2"));
+//		restored1.add("3");
+//		assertThat(state.get(), containsInAnyOrder("2", "3"));
+//
+//		backend.dispose();
+//	}
+
+	/**
+	 * Different "personalities" of {@link CustomStringSerializer}. Instead of creating
+	 * different classes we parameterize the serializer with this and
+	 * {@link CustomStringSerializerConfigSnapshot} will instantiate serializers with the correct
+	 * personality.
+	 */
+	public enum SerializerVersion {
+		INITIAL,
+		RECONFIGURED,
+		RESTORE,
+		NEW
+	}
+
+	/**
+	 * The compatibility behaviour of {@link CustomStringSerializer}. This controls what
+	 * type of serializer {@link CustomStringSerializerConfigSnapshot} will create for
+	 * the different methods that return/create serializers.
+	 */
+	public enum SerializerCompatibilityType {
+		COMPATIBLE_AS_IS,
+		COMPATIBLE_AFTER_RECONFIGURATION,
+		INCOMPATIBLE
+	}
+
+	public static class CustomStringSerializer extends TypeSerializerSingleton<String> {
+		private static final long serialVersionUID = 1L;
+
+		private static final String EMPTY = "";
+
+		private SerializerCompatibilityType compatibilityType;
+		private SerializerVersion serializerVersion;
+
+		// for counting how often the methods were called from serializers of the different
+		// personalities
+		public static Map<SerializerVersion, Integer> serializeCalled = new HashMap<>();
+		public static Map<SerializerVersion, Integer> deserializeCalled = new HashMap<>();
+
+		static void resetCountingMaps() {
+			serializeCalled = new HashMap<>();
+			deserializeCalled = new HashMap<>();
+		}
+
+		CustomStringSerializer(
+			SerializerCompatibilityType compatibilityType,
+			SerializerVersion serializerVersion) {
+			this.compatibilityType = compatibilityType;
+			this.serializerVersion = serializerVersion;
+		}
+
+		@Override
+		public boolean isImmutableType() {
+			return true;
+		}
+
+		@Override
+		public String createInstance() {
+			return EMPTY;
+		}
+
+		@Override
+		public String copy(String from) {
+			return from;
+		}
+
+		@Override
+		public String copy(String from, String reuse) {
+			return from;
+		}
+
+		@Override
+		public int getLength() {
+			return -1;
+		}
+
+		@Override
+		public void serialize(String record, DataOutputView target) throws IOException {
+			System.out.println("SER " + serializerVersion);
+			serializeCalled.compute(serializerVersion, (k, v) -> v == null ? 1 : v + 1);
+			StringValue.writeString(record, target);
+		}
+
+		@Override
+		public String deserialize(DataInputView source) throws IOException {
+			System.out.println("DESER " + serializerVersion);
+			deserializeCalled.compute(serializerVersion, (k, v) -> v == null ? 1 : v + 1);
+			return StringValue.readString(source);
+		}
+
+		@Override
+		public String deserialize(String record, DataInputView source) throws IOException {
+			System.out.println("DESER " + serializerVersion);
+			deserializeCalled.compute(serializerVersion, (k, v) -> v == null ? 1 : v + 1);
+			return deserialize(source);
+		}
+
+		@Override
+		public void copy(DataInputView source, DataOutputView target) throws IOException {
+			StringValue.copyString(source, target);
+		}
+
+		@Override
+		public boolean canEqual(Object obj) {
+			return obj instanceof StateBackendMigrationTestBase.CustomStringSerializer;
+		}
+
+		@Override
+		protected boolean isCompatibleSerializationFormatIdentifier(String identifier) {
+			return super.isCompatibleSerializationFormatIdentifier(identifier)
+				|| identifier.equals(StringValue.class.getCanonicalName());
+		}
+
+		@Override
+		public TypeSerializerConfigSnapshot<String> snapshotConfiguration() {
+			return new CustomStringSerializerConfigSnapshot(compatibilityType);
+		}
+	}
+
+	public static class CustomStringSerializerConfigSnapshot extends TypeSerializerConfigSnapshot<String> {
+
+		private SerializerCompatibilityType compatibilityType;
+
+		public CustomStringSerializerConfigSnapshot() {
+		}
+
+		public CustomStringSerializerConfigSnapshot(SerializerCompatibilityType compatibilityType) {
+			this.compatibilityType = compatibilityType;
+		}
+
+		@Override
+		public void write(DataOutputView out) throws IOException {
+			super.write(out);
+			System.out.println("WRITE SNAPSHOT: " + compatibilityType);
+			out.writeUTF(compatibilityType.toString());
+		}
+
+		@Override
+		public void read(DataInputView in) throws IOException {
+			super.read(in);
+			compatibilityType = SerializerCompatibilityType.valueOf(in.readUTF());
+			System.out.println("READ SNAPSHOT: " + compatibilityType);
+		}
+
+		@Override
+		public TypeSerializer<String> restoreSerializer() {
+			if (compatibilityType == SerializerCompatibilityType.COMPATIBLE_AS_IS) {
+				return new CustomStringSerializer(compatibilityType, SerializerVersion.NEW);
+			} else {
+				return new CustomStringSerializer(compatibilityType, SerializerVersion.RESTORE);
+			}
+		}
+
+		@Override
+		public TypeSerializerSchemaCompatibility<String> resolveSchemaCompatibility(TypeSerializer<?> newSerializer) {
+			if (compatibilityType == SerializerCompatibilityType.COMPATIBLE_AFTER_RECONFIGURATION) {
+				System.out.println(compatibilityType);
+				return TypeSerializerSchemaCompatibility.compatibleAfterReconfiguration(
+					new CustomStringSerializer(compatibilityType, SerializerVersion.RECONFIGURED));
+			} else {
+				System.out.println(compatibilityType);
+				return TypeSerializerSchemaCompatibility.incompatible();
+			}
+		}
+
+		@Override
+		public boolean equals(Object obj) {
+			return obj instanceof CustomStringSerializerConfigSnapshot;
+		}
+
+		@Override
+		public int hashCode() {
+			return 0;
+		}
+
+		@Override
+		public int getVersion() {
+			return 0;
+		}
+	}
+
+	public static class CustomVoidNamespaceSerializer extends TypeSerializerSingleton<VoidNamespace> {
+		private static final long serialVersionUID = 1L;
+
+		public static final CustomVoidNamespaceSerializer INSTANCE = new CustomVoidNamespaceSerializer();
+
+		@Override
+		public boolean isImmutableType() {
+			return true;
+		}
+
+		@Override
+		public VoidNamespace createInstance() {
+			return VoidNamespace.get();
+		}
+
+		@Override
+		public VoidNamespace copy(VoidNamespace from) {
+			return VoidNamespace.get();
+		}
+
+		@Override
+		public VoidNamespace copy(VoidNamespace from, VoidNamespace reuse) {
+			return VoidNamespace.get();
+		}
+
+		@Override
+		public int getLength() {
+			return 0;
+		}
+
+		@Override
+		public void serialize(VoidNamespace record, DataOutputView target) throws IOException {
+			// Make progress in the stream, write one byte.
+			//
+			// We could just skip writing anything here, because of the way this is
+			// used with the state backends, but if it is ever used somewhere else
+			// (even though it is unlikely to happen), it would be a problem.
+			target.write(0);
+		}
+
+		@Override
+		public VoidNamespace deserialize(DataInputView source) throws IOException {
+			source.readByte();
+			return VoidNamespace.get();
+		}
+
+		@Override
+		public VoidNamespace deserialize(VoidNamespace reuse, DataInputView source) throws IOException {
+			source.readByte();
+			return VoidNamespace.get();
+		}
+
+		@Override
+		public void copy(DataInputView source, DataOutputView target) throws IOException {
+			target.write(source.readByte());
+		}
+
+		@Override
+		public boolean canEqual(Object obj) {
+			return obj instanceof CustomVoidNamespaceSerializer;
+		}
+
+		@Override
+		public TypeSerializerConfigSnapshot<VoidNamespace> snapshotConfiguration() {
+			return new CustomVoidNamespaceSerializerConfigSnapshot();
+		}
+	}
+
+	public static class CustomVoidNamespaceSerializerConfigSnapshot extends TypeSerializerConfigSnapshot<VoidNamespace> {
+
+		@Override
+		public TypeSerializer<VoidNamespace> restoreSerializer() {
+			return new CustomVoidNamespaceSerializer();
+		}
+
+		@Override
+		public TypeSerializerSchemaCompatibility<VoidNamespace> resolveSchemaCompatibility(TypeSerializer<?> newSerializer) {
+			return TypeSerializerSchemaCompatibility.compatibleAsIs();
+		}
+
+		@Override
+		public boolean equals(Object obj) {
+			return obj instanceof CustomVoidNamespaceSerializerConfigSnapshot;
+		}
+
+		@Override
+		public int hashCode() {
+			return 0;
+		}
+
+		@Override
+		public int getVersion() {
+			return 0;
+		}
+	}
+}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
index ad67171a01c..e48fd573f94 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/StateBackendTestBase.java
@@ -36,7 +36,7 @@
 import org.apache.flink.api.common.state.ValueState;
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.ParameterlessTypeSerializerConfig;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
@@ -4242,19 +4242,19 @@ private TestReconfigurableCustomTypeSerializer(boolean reconfigured) {
 		}
 
 		@Override
-		public TypeSerializerConfigSnapshot snapshotConfiguration() {
-			return new ParameterlessTypeSerializerConfig(getClass().getName());
+		public TypeSerializerConfigSnapshot<TestCustomStateClass> snapshotConfiguration() {
+			return new ParameterlessTypeSerializerConfig<>(getClass().getName());
 		}
 
 		@Override
-		public CompatibilityResult<TestCustomStateClass> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+		public TypeSerializerSchemaCompatibility<TestCustomStateClass> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 			if (configSnapshot instanceof ParameterlessTypeSerializerConfig &&
-					((ParameterlessTypeSerializerConfig) configSnapshot).getSerializationFormatIdentifier().equals(getClass().getName())) {
+					((ParameterlessTypeSerializerConfig<?>) configSnapshot).getSerializationFormatIdentifier().equals(getClass().getName())) {
 
 				this.reconfigured = true;
-				return CompatibilityResult.compatible();
+				return TypeSerializerSchemaCompatibility.compatibleAsIs();
 			} else {
-				return CompatibilityResult.requiresMigration();
+				return TypeSerializerSchemaCompatibility.incompatible();
 			}
 		}
 
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java
index 4f36d625922..5df2fa19886 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java
@@ -19,7 +19,7 @@
 package org.apache.flink.runtime.state.heap;
 
 import org.apache.flink.api.common.state.StateDescriptor;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.api.common.typeutils.base.IntSerializer;
@@ -654,12 +654,12 @@ public void disable() {
 		}
 
 		@Override
-		public TypeSerializerConfigSnapshot snapshotConfiguration() {
+		public TypeSerializerConfigSnapshot<Integer> snapshotConfiguration() {
 			throw new UnsupportedOperationException();
 		}
 
 		@Override
-		public CompatibilityResult<Integer> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+		public TypeSerializerSchemaCompatibility<Integer> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 			throw new UnsupportedOperationException();
 		}
 	}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/recordutils/RecordSerializer.java b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/recordutils/RecordSerializer.java
index ce23f30006b..5ed6ed9cd2a 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/recordutils/RecordSerializer.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/recordutils/RecordSerializer.java
@@ -21,7 +21,7 @@
 
 import java.io.IOException;
 
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 import org.apache.flink.core.memory.DataInputView;
@@ -145,12 +145,12 @@ public int hashCode() {
 	}
 
 	@Override
-	public TypeSerializerConfigSnapshot snapshotConfiguration() {
+	public TypeSerializerConfigSnapshot<Record> snapshotConfiguration() {
 		throw new UnsupportedOperationException();
 	}
 
 	@Override
-	public CompatibilityResult<Record> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+	public TypeSerializerSchemaCompatibility<Record> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 		throw new UnsupportedOperationException();
 	}
 }
diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerConfigSnapshot.java b/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerConfigSnapshot.java
new file mode 100644
index 00000000000..cd3d4494e17
--- /dev/null
+++ b/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerConfigSnapshot.java
@@ -0,0 +1,59 @@
+/*
+ * 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.api.scala.typeutils;
+
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+
+import scala.util.Either;
+
+/**
+ * Configuration snapshot for serializers of Scala's {@link Either} type,
+ * containing configuration snapshots of the Left and Right serializers.
+ */
+public class ScalaEitherSerializerConfigSnapshot<E extends Either<L, R>, L, R>
+		extends CompositeTypeSerializerConfigSnapshot<E> {
+
+	private static final int VERSION = 1;
+
+	/** This empty nullary constructor is required for deserializing the configuration. */
+	public ScalaEitherSerializerConfigSnapshot() {}
+
+	public ScalaEitherSerializerConfigSnapshot(TypeSerializer<L> leftSerializer, TypeSerializer<R> rightSerializer) {
+		super(leftSerializer, rightSerializer);
+	}
+
+	@Override
+	public int getVersion() {
+		return VERSION;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	protected TypeSerializer<E> restoreSerializer(TypeSerializer<?>... restoredNestedSerializers) {
+		return new EitherSerializer<>(
+			(TypeSerializer<L>) restoredNestedSerializers[0],
+			(TypeSerializer<R>) restoredNestedSerializers[1]);
+	}
+
+	@Override
+	protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+		return newSerializer instanceof EitherSerializer;
+	}
+}
diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaOptionSerializerConfigSnapshot.java b/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaOptionSerializerConfigSnapshot.java
index 03eef12e9ca..7a87f8b6f57 100644
--- a/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaOptionSerializerConfigSnapshot.java
+++ b/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaOptionSerializerConfigSnapshot.java
@@ -22,6 +22,8 @@
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 
+import scala.Option;
+
 /**
  * A {@link TypeSerializerConfigSnapshot} for the Scala {@link OptionSerializer}.
  *
@@ -29,9 +31,9 @@
  * allow calling different base class constructors from subclasses, while we need that
  * for the default empty constructor.
  */
-public final class ScalaOptionSerializerConfigSnapshot<E> extends CompositeTypeSerializerConfigSnapshot {
+public final class ScalaOptionSerializerConfigSnapshot<E> extends CompositeTypeSerializerConfigSnapshot<Option<E>> {
 
-	private static final int VERSION = 1;
+	private static final int VERSION = 2;
 
 	/** This empty nullary constructor is required for deserializing the configuration. */
 	public ScalaOptionSerializerConfigSnapshot() {}
@@ -44,4 +46,25 @@ public ScalaOptionSerializerConfigSnapshot(TypeSerializer<E> elementSerializer)
 	public int getVersion() {
 		return VERSION;
 	}
+
+	@Override
+	public int[] getCompatibleVersions() {
+		return new int[]{VERSION, 1};
+	}
+
+	@Override
+	protected boolean containsSerializers() {
+		return getReadVersion() < 2;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	protected TypeSerializer<Option<E>> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+		return new OptionSerializer<>((TypeSerializer<E>) restoredNestedSerializers[0]);
+	}
+
+	@Override
+	protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+		return newSerializer instanceof OptionSerializer;
+	}
 }
diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaTrySerializerConfigSnapshot.java b/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaTrySerializerConfigSnapshot.java
index 6abb3ea06d7..b4118f946ca 100644
--- a/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaTrySerializerConfigSnapshot.java
+++ b/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaTrySerializerConfigSnapshot.java
@@ -22,6 +22,8 @@
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 
+import scala.util.Try;
+
 /**
  * A {@link TypeSerializerConfigSnapshot} for the Scala {@link TrySerializer}.
  *
@@ -29,9 +31,9 @@
  * allow calling different base class constructors from subclasses, while we need that
  * for the default empty constructor.
  */
-public class ScalaTrySerializerConfigSnapshot<E> extends CompositeTypeSerializerConfigSnapshot {
+public class ScalaTrySerializerConfigSnapshot<E> extends CompositeTypeSerializerConfigSnapshot<Try<E>> {
 
-	private static final int VERSION = 1;
+	private static final int VERSION = 2;
 
 	/** This empty nullary constructor is required for deserializing the configuration. */
 	public ScalaTrySerializerConfigSnapshot() {}
@@ -47,4 +49,27 @@ public ScalaTrySerializerConfigSnapshot(
 	public int getVersion() {
 		return VERSION;
 	}
+
+	@Override
+	public int[] getCompatibleVersions() {
+		return new int[]{VERSION, 1};
+	}
+
+	@Override
+	protected boolean containsSerializers() {
+		return getReadVersion() < 2;
+	}
+
+	@SuppressWarnings("unchecked")
+	@Override
+	protected TypeSerializer<Try<E>> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+		return new TrySerializer<>(
+			(TypeSerializer<E>) restoredNestedSerializers[0],
+			(TypeSerializer<Throwable>) restoredNestedSerializers[1]);
+	}
+
+	@Override
+	protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+		return newSerializer instanceof TrySerializer;
+	}
 }
diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/TraversableSerializerConfigSnapshot.java b/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/TraversableSerializerConfigSnapshot.java
index 9a39421dad5..8196c5041a2 100644
--- a/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/TraversableSerializerConfigSnapshot.java
+++ b/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/TraversableSerializerConfigSnapshot.java
@@ -22,6 +22,8 @@
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
 
+import scala.collection.TraversableOnce;
+
 /**
  * A {@link TypeSerializerConfigSnapshot} for the Scala {@link TraversableSerializer}.
  *
@@ -29,7 +31,8 @@
  * allow calling different base class constructors from subclasses, while we need that
  * for the default empty constructor.
  */
-public class TraversableSerializerConfigSnapshot<E> extends CompositeTypeSerializerConfigSnapshot {
+public class TraversableSerializerConfigSnapshot<T extends TraversableOnce<E>, E>
+		extends CompositeTypeSerializerConfigSnapshot<T> {
 
 	private static final int VERSION = 1;
 
@@ -44,4 +47,15 @@ public TraversableSerializerConfigSnapshot(TypeSerializer<E> elementSerializer)
 	public int getVersion() {
 		return VERSION;
 	}
+
+	@Override
+	protected TypeSerializer<T> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+		// TODO to be implemented in follow-up commits
+		return null;
+	}
+
+	@Override
+	protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+		return newSerializer instanceof TraversableSerializer;
+	}
 }
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
index 74f5f5bef8f..5338afbee71 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
@@ -149,15 +149,6 @@ private[flink] trait TypeInformationGen[C <: Context] {
             override def createInstance(fields: Array[AnyRef]): T = {
               instance.splice
             }
-
-            override def createSerializerInstance(
-                tupleClass: Class[T],
-                fieldSerializers: Array[TypeSerializer[_]]) = {
-              this.getClass
-                .getConstructors()(0)
-                .newInstance(tupleClass, fieldSerializers)
-                .asInstanceOf[CaseClassSerializer[T]]
-            }
           }
         }
       }
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
index 4fd559ad6ed..0832b69625c 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
@@ -118,11 +118,5 @@ package object scala {
     override def createInstance(fields: Array[AnyRef]) = {
       (fields(0).asInstanceOf[T1], fields(1).asInstanceOf[T2])
     }
-
-    override def createSerializerInstance(
-        tupleClass: Class[(T1, T2)],
-        fieldSerializers: Array[TypeSerializer[_]]) = {
-      new Tuple2CaseClassSerializer[T1, T2](tupleClass, fieldSerializers)
-    }
   }
 }
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala
index c0599138be5..892cb1dab73 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/CaseClassSerializer.scala
@@ -32,7 +32,9 @@ import org.apache.flink.types.NullFieldException
 abstract class CaseClassSerializer[T <: Product](
     clazz: Class[T],
     scalaFieldSerializers: Array[TypeSerializer[_]])
-  extends TupleSerializerBase[T](clazz, scalaFieldSerializers)
+  // TODO configuration snapshot of case class serializers is to
+  // TODO be implemented in follow-up commits
+  extends TupleSerializerBase[T](clazz, null, scalaFieldSerializers)
   with Cloneable {
 
   @transient var fields : Array[AnyRef] = _
@@ -81,15 +83,6 @@ abstract class CaseClassSerializer[T <: Product](
     createInstance(fields)
   }
 
-  override def createSerializerInstance(
-      tupleClass: Class[T],
-      fieldSerializers: Array[TypeSerializer[_]]): TupleSerializerBase[T] = {
-    this.getClass
-      .getConstructors()(0)
-      .newInstance(tupleClass, fieldSerializers)
-      .asInstanceOf[CaseClassSerializer[T]]
-  }
-
   def copy(from: T, reuse: T): T = {
     copy(from)
   }
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala
index 439e0c2865d..4e61730b526 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EitherSerializer.scala
@@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal
 import org.apache.flink.api.common.typeutils._
 import org.apache.flink.api.java.typeutils.runtime.EitherSerializerConfigSnapshot
 import org.apache.flink.core.memory.{DataInputView, DataOutputView}
+import org.apache.flink.util.Preconditions
 
 /**
  * Serializer for [[Either]].
@@ -30,7 +31,12 @@ import org.apache.flink.core.memory.{DataInputView, DataOutputView}
 class EitherSerializer[A, B, T <: Either[A, B]](
     val leftSerializer: TypeSerializer[A],
     val rightSerializer: TypeSerializer[B])
-  extends TypeSerializer[T] {
+  extends CompositeTypeSerializer[T](
+    new ScalaEitherSerializerConfigSnapshot[T, A, B](
+      Preconditions.checkNotNull(leftSerializer),
+      Preconditions.checkNotNull(rightSerializer)),
+    leftSerializer,
+    rightSerializer) {
 
   override def duplicate: EitherSerializer[A,B,T] = this
 
@@ -111,51 +117,13 @@ class EitherSerializer[A, B, T <: Either[A, B]](
   // Serializer configuration snapshotting & compatibility
   // --------------------------------------------------------------------------------------------
 
-  override def snapshotConfiguration(): EitherSerializerConfigSnapshot[A, B] = {
-    new EitherSerializerConfigSnapshot[A, B](leftSerializer, rightSerializer)
-  }
+  override def isComparableSnapshot(
+      configSnapshot: TypeSerializerConfigSnapshot[_]): Boolean = {
 
-  override def ensureCompatibility(
-      configSnapshot: TypeSerializerConfigSnapshot): CompatibilityResult[T] = {
-
-    configSnapshot match {
-      case eitherSerializerConfig: EitherSerializerConfigSnapshot[A, B] =>
-        val previousLeftRightSerWithConfigs =
-          eitherSerializerConfig.getNestedSerializersAndConfigs
-
-        val leftCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-          previousLeftRightSerWithConfigs.get(0).f0,
-          classOf[UnloadableDummyTypeSerializer[_]],
-          previousLeftRightSerWithConfigs.get(0).f1,
-          leftSerializer)
-
-        val rightCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-          previousLeftRightSerWithConfigs.get(1).f0,
-          classOf[UnloadableDummyTypeSerializer[_]],
-          previousLeftRightSerWithConfigs.get(1).f1,
-          rightSerializer)
-
-        if (leftCompatResult.isRequiresMigration
-            || rightCompatResult.isRequiresMigration) {
-
-          if (leftCompatResult.getConvertDeserializer != null
-              && rightCompatResult.getConvertDeserializer != null) {
-
-            CompatibilityResult.requiresMigration(
-              new EitherSerializer[A, B, T](
-                new TypeDeserializerAdapter(leftCompatResult.getConvertDeserializer),
-                new TypeDeserializerAdapter(rightCompatResult.getConvertDeserializer)
-              )
-            )
-
-          } else {
-            CompatibilityResult.requiresMigration()
-          }
-        } else {
-          CompatibilityResult.compatible()
-        }
-
-      case _ => CompatibilityResult.requiresMigration()
-    }
+    configSnapshot.isInstanceOf[ScalaEitherSerializerConfigSnapshot[T, A, B]] ||
+      // backwards compatibility path;
+      // Flink versions older or equal to 1.5.x uses a
+      // EitherSerializerConfigSnapshot as the snapshot
+      configSnapshot.isInstanceOf[EitherSerializerConfigSnapshot[A, B]]
   }
 }
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala
index abc56d133e0..6d342155e22 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala
@@ -20,7 +20,7 @@ package org.apache.flink.api.scala.typeutils
 import java.io.IOException
 
 import org.apache.flink.annotation.Internal
-import org.apache.flink.api.common.typeutils.{CompatibilityResult, TypeSerializer, TypeSerializerConfigSnapshot}
+import org.apache.flink.api.common.typeutils.{TypeSerializer, TypeSerializerConfigSnapshot, TypeSerializerSchemaCompatibility}
 import org.apache.flink.api.common.typeutils.base.IntSerializer
 import org.apache.flink.api.java.typeutils.runtime.{DataInputViewStream, DataOutputViewStream}
 import org.apache.flink.core.memory.{DataInputView, DataOutputView}
@@ -84,7 +84,8 @@ class EnumValueSerializer[E <: Enumeration](val enum: E) extends TypeSerializer[
   }
 
   override def ensureCompatibility(
-      configSnapshot: TypeSerializerConfigSnapshot): CompatibilityResult[E#Value] = {
+      configSnapshot: TypeSerializerConfigSnapshot[_]):
+  TypeSerializerSchemaCompatibility[E#Value] = {
 
     configSnapshot match {
       case enumSerializerConfigSnapshot: EnumValueSerializer.ScalaEnumSerializerConfigSnapshot[_] =>
@@ -99,22 +100,22 @@ class EnumValueSerializer[E <: Enumeration](val enum: E) extends TypeSerializer[
             } catch {
               case _: NoSuchElementException =>
                 // couldn't find an enum value for the given index
-                return CompatibilityResult.requiresMigration()
+                return TypeSerializerSchemaCompatibility.incompatible()
             }
 
             if (!previousEnumConstant.equals(enumValue.toString)) {
               // compatible only if new enum constants are only appended,
               // and original constants must be in the exact same order
-              return CompatibilityResult.requiresMigration()
+              return TypeSerializerSchemaCompatibility.incompatible()
             }
           }
 
-          CompatibilityResult.compatible()
+          TypeSerializerSchemaCompatibility.compatibleAsIs()
         } else {
-          CompatibilityResult.requiresMigration()
+          TypeSerializerSchemaCompatibility.incompatible()
         }
 
-      case _ => CompatibilityResult.requiresMigration()
+      case _ => TypeSerializerSchemaCompatibility.incompatible()
     }
   }
 }
@@ -122,7 +123,7 @@ class EnumValueSerializer[E <: Enumeration](val enum: E) extends TypeSerializer[
 object EnumValueSerializer {
 
   class ScalaEnumSerializerConfigSnapshot[E <: Enumeration]
-      extends TypeSerializerConfigSnapshot {
+      extends TypeSerializerConfigSnapshot[E#Value] {
 
     var enumClass: Class[E] = _
     var enumConstants: List[(String, Int)] = _
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala
index 01ca29594e2..806885931b1 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala
@@ -18,7 +18,7 @@
 package org.apache.flink.api.scala.typeutils
 
 import org.apache.flink.annotation.Internal
-import org.apache.flink.api.common.typeutils.{CompatibilityResult, TypeSerializer, TypeSerializerConfigSnapshot}
+import org.apache.flink.api.common.typeutils.{TypeSerializerSchemaCompatibility, TypeSerializer, TypeSerializerConfigSnapshot}
 import org.apache.flink.core.memory.{DataInputView, DataOutputView}
 
 /**
@@ -56,11 +56,11 @@ class NothingSerializer extends TypeSerializer[Any] {
   override def deserialize(reuse: Any, source: DataInputView): Any =
     throw new RuntimeException("This must not be used. You encountered a bug.")
 
-  override def snapshotConfiguration(): TypeSerializerConfigSnapshot =
+  override def snapshotConfiguration(): TypeSerializerConfigSnapshot[Any] =
     throw new RuntimeException("This must not be used. You encountered a bug.")
 
   override def ensureCompatibility(
-      configSnapshot: TypeSerializerConfigSnapshot): CompatibilityResult[Any] =
+      configSnapshot: TypeSerializerConfigSnapshot[_]): TypeSerializerSchemaCompatibility[Any] =
     throw new RuntimeException("This must not be used. You encountered a bug.")
 
   override def equals(obj: Any): Boolean = {
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionSerializer.scala
index aa4a0ea75b8..a2644d09266 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionSerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionSerializer.scala
@@ -20,6 +20,7 @@ package org.apache.flink.api.scala.typeutils
 import org.apache.flink.annotation.Internal
 import org.apache.flink.api.common.typeutils._
 import org.apache.flink.core.memory.{DataInputView, DataOutputView}
+import org.apache.flink.util.Preconditions
 
 /**
  * Serializer for [[Option]].
@@ -27,7 +28,9 @@ import org.apache.flink.core.memory.{DataInputView, DataOutputView}
 @Internal
 @SerialVersionUID(-8635243274072627338L)
 class OptionSerializer[A](val elemSerializer: TypeSerializer[A])
-  extends TypeSerializer[Option[A]] {
+  extends CompositeTypeSerializer[Option[A]](
+    new ScalaOptionSerializerConfigSnapshot[A](Preconditions.checkNotNull(elemSerializer)),
+    elemSerializer) {
 
   override def duplicate: OptionSerializer[A] = {
     val duplicatedElemSerializer = elemSerializer.duplicate()
@@ -97,49 +100,11 @@ class OptionSerializer[A](val elemSerializer: TypeSerializer[A])
     elemSerializer.hashCode()
   }
 
-  // --------------------------------------------------------------------------------------------
-  // Serializer configuration snapshotting & compatibility
-  // --------------------------------------------------------------------------------------------
+  override def isComparableSnapshot(
+      configSnapshot: TypeSerializerConfigSnapshot[_]): Boolean = {
 
-  override def snapshotConfiguration(): ScalaOptionSerializerConfigSnapshot[A] = {
-    new ScalaOptionSerializerConfigSnapshot[A](elemSerializer)
-  }
-
-  override def ensureCompatibility(
-      configSnapshot: TypeSerializerConfigSnapshot): CompatibilityResult[Option[A]] = {
-
-    configSnapshot match {
-      case optionSerializerConfigSnapshot
-          : ScalaOptionSerializerConfigSnapshot[A] =>
-        ensureCompatibility(optionSerializerConfigSnapshot)
-      case legacyOptionSerializerConfigSnapshot
-          : OptionSerializer.OptionSerializerConfigSnapshot[A] =>
-        ensureCompatibility(legacyOptionSerializerConfigSnapshot)
-      case _ => CompatibilityResult.requiresMigration()
-    }
-  }
-
-  private def ensureCompatibility(
-      compositeConfigSnapshot: CompositeTypeSerializerConfigSnapshot)
-      : CompatibilityResult[Option[A]] = {
-
-    val compatResult = CompatibilityUtil.resolveCompatibilityResult(
-      compositeConfigSnapshot.getSingleNestedSerializerAndConfig.f0,
-      classOf[UnloadableDummyTypeSerializer[_]],
-      compositeConfigSnapshot.getSingleNestedSerializerAndConfig.f1,
-      elemSerializer)
-
-    if (compatResult.isRequiresMigration) {
-      if (compatResult.getConvertDeserializer != null) {
-        CompatibilityResult.requiresMigration(
-          new OptionSerializer[A](
-            new TypeDeserializerAdapter(compatResult.getConvertDeserializer)))
-      } else {
-        CompatibilityResult.requiresMigration()
-      }
-    } else {
-      CompatibilityResult.compatible()
-    }
+    configSnapshot.isInstanceOf[ScalaOptionSerializerConfigSnapshot[A]] ||
+      configSnapshot.isInstanceOf[OptionSerializer.OptionSerializerConfigSnapshot[A]]
   }
 }
 
@@ -150,13 +115,34 @@ object OptionSerializer {
     * Once Flink 1.3.x is no longer supported, this can be removed.
     */
   class OptionSerializerConfigSnapshot[A]()
-      extends CompositeTypeSerializerConfigSnapshot {
+      extends CompositeTypeSerializerConfigSnapshot[Option[A]] {
 
     override def getVersion: Int = OptionSerializerConfigSnapshot.VERSION
+
+    override def restoreSerializer(
+        restoredNestedSerializers: TypeSerializer[_]*
+      ): TypeSerializer[Option[A]] = {
+
+      new OptionSerializer[A](
+        restoredNestedSerializers(0).asInstanceOf[TypeSerializer[A]])
+    }
+
+    override def containsSerializers(): Boolean = {
+      getReadVersion < 2
+    }
+
+    override def isRecognizableSerializer(
+        newSerializer: TypeSerializer[_]): Boolean = {
+      newSerializer.isInstanceOf[OptionSerializer[A]]
+    }
+
+    override def getCompatibleVersions: Array[Int] = {
+      Array(getVersion, 1)
+    }
   }
 
   object OptionSerializerConfigSnapshot {
-    val VERSION = 1
+    val VERSION = 2
   }
 
 }
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
index b54193b4829..90879ec5b37 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TraversableSerializer.scala
@@ -152,30 +152,28 @@ abstract class TraversableSerializer[T <: TraversableOnce[E], E](
     obj.isInstanceOf[TraversableSerializer[_, _]]
   }
 
-  override def snapshotConfiguration(): TraversableSerializerConfigSnapshot[E] = {
-    new TraversableSerializerConfigSnapshot[E](elementSerializer)
+  override def snapshotConfiguration(): TraversableSerializerConfigSnapshot[T, E] = {
+    new TraversableSerializerConfigSnapshot[T, E](elementSerializer)
   }
 
   override def ensureCompatibility(
-      configSnapshot: TypeSerializerConfigSnapshot): CompatibilityResult[T] = {
+      configSnapshot: TypeSerializerConfigSnapshot[_]): TypeSerializerSchemaCompatibility[T] = {
 
     configSnapshot match {
       case traversableSerializerConfigSnapshot
-          : TraversableSerializerConfigSnapshot[E] =>
+          : TraversableSerializerConfigSnapshot[T, E] =>
 
         val elemCompatRes = CompatibilityUtil.resolveCompatibilityResult(
-          traversableSerializerConfigSnapshot.getSingleNestedSerializerAndConfig.f0,
-          classOf[UnloadableDummyTypeSerializer[_]],
-          traversableSerializerConfigSnapshot.getSingleNestedSerializerAndConfig.f1,
+          traversableSerializerConfigSnapshot.getNestedSerializerConfigSnapshot(0),
           elementSerializer)
 
-        if (elemCompatRes.isRequiresMigration) {
-          CompatibilityResult.requiresMigration()
+        if (elemCompatRes.isIncompatible) {
+          TypeSerializerSchemaCompatibility.incompatible()
         } else {
-          CompatibilityResult.compatible()
+          TypeSerializerSchemaCompatibility.compatibleAsIs()
         }
 
-      case _ => CompatibilityResult.requiresMigration()
+      case _ => TypeSerializerSchemaCompatibility.incompatible()
     }
   }
 }
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala
index cc9c5ccede7..174dea56f0f 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TrySerializer.scala
@@ -18,10 +18,9 @@
 package org.apache.flink.api.scala.typeutils
 
 import org.apache.flink.annotation.Internal
-import org.apache.flink.api.common.ExecutionConfig
 import org.apache.flink.api.common.typeutils._
-import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer
 import org.apache.flink.core.memory.{DataInputView, DataOutputView}
+import org.apache.flink.util.Preconditions
 
 import scala.util.{Failure, Success, Try}
 
@@ -32,13 +31,16 @@ import scala.util.{Failure, Success, Try}
 @SerialVersionUID(-3052182891252564491L)
 class TrySerializer[A](
     private val elemSerializer: TypeSerializer[A],
-    private val executionConfig: ExecutionConfig)
-  extends TypeSerializer[Try[A]] {
+    private val throwableSerializer: TypeSerializer[Throwable])
+  extends CompositeTypeSerializer[Try[A]](
+    new ScalaTrySerializerConfigSnapshot[A](
+      Preconditions.checkNotNull(elemSerializer),
+      Preconditions.checkNotNull(throwableSerializer)),
+    elemSerializer,
+    throwableSerializer) {
 
   override def duplicate: TrySerializer[A] = this
 
-  val throwableSerializer = new KryoSerializer[Throwable](classOf[Throwable], executionConfig)
-
   override def createInstance: Try[A] = {
     Failure(new RuntimeException("Empty Failure"))
   }
@@ -97,55 +99,20 @@ class TrySerializer[A](
   }
 
   override def hashCode(): Int = {
-    31 * elemSerializer.hashCode() + executionConfig.hashCode()
+    31 * elemSerializer.hashCode() + throwableSerializer.hashCode()
   }
 
   // --------------------------------------------------------------------------------------------
   // Serializer configuration snapshotting & compatibility
   // --------------------------------------------------------------------------------------------
 
-  override def snapshotConfiguration(): ScalaTrySerializerConfigSnapshot[A] = {
-    new ScalaTrySerializerConfigSnapshot[A](elemSerializer, throwableSerializer)
-  }
-
-  override def ensureCompatibility(
-      configSnapshot: TypeSerializerConfigSnapshot): CompatibilityResult[Try[A]] = {
-
-    configSnapshot match {
-      case trySerializerConfigSnapshot
-          : ScalaTrySerializerConfigSnapshot[A] =>
-        ensureCompatibility(trySerializerConfigSnapshot)
-      case legacyTrySerializerConfigSnapshot
-          : TrySerializer.TrySerializerConfigSnapshot[A] =>
-        ensureCompatibility(legacyTrySerializerConfigSnapshot)
-      case _ => CompatibilityResult.requiresMigration()
-    }
-  }
-
-  private def ensureCompatibility(
-      compositeConfigSnapshot: CompositeTypeSerializerConfigSnapshot)
-        : CompatibilityResult[Try[A]] = {
-
-    val previousSerializersAndConfigs =
-      compositeConfigSnapshot.getNestedSerializersAndConfigs
-
-    val elemCompatRes = CompatibilityUtil.resolveCompatibilityResult(
-      previousSerializersAndConfigs.get(0).f0,
-      classOf[UnloadableDummyTypeSerializer[_]],
-      previousSerializersAndConfigs.get(0).f1,
-      elemSerializer)
-
-    val throwableCompatRes = CompatibilityUtil.resolveCompatibilityResult(
-      previousSerializersAndConfigs.get(1).f0,
-      classOf[UnloadableDummyTypeSerializer[_]],
-      previousSerializersAndConfigs.get(1).f1,
-      throwableSerializer)
-
-    if (elemCompatRes.isRequiresMigration || throwableCompatRes.isRequiresMigration) {
-      CompatibilityResult.requiresMigration()
-    } else {
-      CompatibilityResult.compatible()
-    }
+  override def isComparableSnapshot(
+      configSnapshot: TypeSerializerConfigSnapshot[_]): Boolean = {
+    configSnapshot.isInstanceOf[ScalaTrySerializerConfigSnapshot[A]] ||
+      // backwards compatibility path;
+      // Flink versions older or equal to 1.5.x returns a
+      // TrySerializer.TrySerializerConfigSnapshot as the snapshot
+      configSnapshot.isInstanceOf[TrySerializer.TrySerializerConfigSnapshot[A]]
   }
 }
 
@@ -156,13 +123,31 @@ object TrySerializer {
     * Once Flink 1.3.x is no longer supported, this can be removed.
     */
   class TrySerializerConfigSnapshot[A]()
-      extends CompositeTypeSerializerConfigSnapshot() {
+      extends CompositeTypeSerializerConfigSnapshot[Try[A]]() {
 
     override def getVersion: Int = TrySerializerConfigSnapshot.VERSION
+
+    override def restoreSerializer(
+        restoredNestedSerializers: TypeSerializer[_]*
+      ): TypeSerializer[Try[A]] = {
+
+      new TrySerializer[A](
+        restoredNestedSerializers(0).asInstanceOf[TypeSerializer[A]],
+        restoredNestedSerializers(1).asInstanceOf[TypeSerializer[Throwable]])
+    }
+
+    override def containsSerializers(): Boolean = {
+      getReadVersion < 2
+    }
+
+    override def isRecognizableSerializer(
+        newSerializer: TypeSerializer[_]): Boolean = {
+      newSerializer.isInstanceOf[TrySerializer[A]]
+    }
   }
 
   object TrySerializerConfigSnapshot {
-    val VERSION = 1
+    val VERSION = 2
   }
 
 }
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TryTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TryTypeInfo.scala
index b09c353dfab..89c95cbd6bb 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TryTypeInfo.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/TryTypeInfo.scala
@@ -17,13 +17,13 @@
  */
 package org.apache.flink.api.scala.typeutils
 
-import org.apache.flink.annotation.{PublicEvolving, Public}
+import org.apache.flink.annotation.{Public, PublicEvolving}
 import org.apache.flink.api.common.ExecutionConfig
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.common.typeutils.TypeSerializer
+import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer
 
 import scala.collection.JavaConverters._
-
 import scala.util.Try
 
 /**
@@ -52,10 +52,15 @@ class TryTypeInfo[A, T <: Try[A]](val elemTypeInfo: TypeInformation[A])
   def createSerializer(executionConfig: ExecutionConfig): TypeSerializer[T] = {
     if (elemTypeInfo == null) {
       // this happens when the type of a DataSet is None, i.e. DataSet[Failure]
-      new TrySerializer(new NothingSerializer, executionConfig).asInstanceOf[TypeSerializer[T]]
+      new TrySerializer(
+        new NothingSerializer,
+        new KryoSerializer[Throwable](classOf[Throwable], executionConfig)
+      ).asInstanceOf[TypeSerializer[T]]
     } else {
-      new TrySerializer(elemTypeInfo.createSerializer(executionConfig), executionConfig)
-        .asInstanceOf[TypeSerializer[T]]
+      new TrySerializer(
+        elemTypeInfo.createSerializer(executionConfig),
+        new KryoSerializer[Throwable](classOf[Throwable], executionConfig)
+      ).asInstanceOf[TypeSerializer[T]]
     }
   }
 
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerCompatibilityTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerCompatibilityTest.scala
index 7d594fb2b16..7e64dfab0ad 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerCompatibilityTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/runtime/TupleSerializerCompatibilityTest.scala
@@ -59,7 +59,8 @@ class TupleSerializerCompatibilityTest {
 
       val currentSerializer = createTypeInformation[TestCaseClass]
         .createSerializer(new ExecutionConfig())
-      assertFalse(currentSerializer.ensureCompatibility(oldConfigSnapshot).isRequiresMigration)
+      assertFalse(oldConfigSnapshot
+        .resolveSchemaCompatibility(currentSerializer).isIncompatible)
 
       // test old data serialization
       is.close()
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerTest.scala
index f89e9bc4738..56a0f691f2c 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerTest.scala
@@ -35,7 +35,7 @@ class EnumValueSerializerTest extends TestLogger with JUnitSuiteLike {
 
     val snapshot = enumSerializer.snapshotConfiguration()
 
-    assertFalse(enumSerializer.ensureCompatibility(snapshot).isRequiresMigration)
+    assertFalse(snapshot.resolveSchemaCompatibility(enumSerializer).isIncompatible)
   }
 }
 
diff --git a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala
index 8be4debff35..dfbac53ceee 100644
--- a/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala
+++ b/flink-scala/src/test/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializerUpgradeTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.api.scala.typeutils
 import java.io._
 import java.net.{URL, URLClassLoader}
 
-import org.apache.flink.api.common.typeutils.{CompatibilityResult, TypeSerializerSerializationUtil}
+import org.apache.flink.api.common.typeutils.{TypeSerializerSchemaCompatibility, TypeSerializerConfigSnapshotSerializationUtil}
 import org.apache.flink.core.memory.{DataInputViewStreamWrapper, DataOutputViewStreamWrapper}
 import org.apache.flink.util.TestLogger
 import org.junit.rules.TemporaryFolder
@@ -84,7 +84,7 @@ class EnumValueSerializerUpgradeTest extends TestLogger with JUnitSuiteLike {
     */
   @Test
   def checkIdenticalEnums(): Unit = {
-    assertFalse(checkCompatibility(enumA, enumA).isRequiresMigration)
+    assertFalse(checkCompatibility(enumA, enumA).isIncompatible)
   }
 
   /**
@@ -92,7 +92,7 @@ class EnumValueSerializerUpgradeTest extends TestLogger with JUnitSuiteLike {
     */
   @Test
   def checkAppendedField(): Unit = {
-    assertFalse(checkCompatibility(enumA, enumB).isRequiresMigration)
+    assertFalse(checkCompatibility(enumA, enumB).isIncompatible)
   }
 
   /**
@@ -100,7 +100,7 @@ class EnumValueSerializerUpgradeTest extends TestLogger with JUnitSuiteLike {
     */
   @Test
   def checkRemovedField(): Unit = {
-    assertTrue(checkCompatibility(enumA, enumC).isRequiresMigration)
+    assertTrue(checkCompatibility(enumA, enumC).isIncompatible)
   }
 
   /**
@@ -108,7 +108,7 @@ class EnumValueSerializerUpgradeTest extends TestLogger with JUnitSuiteLike {
     */
   @Test
   def checkDifferentFieldOrder(): Unit = {
-    assertTrue(checkCompatibility(enumA, enumD).isRequiresMigration)
+    assertTrue(checkCompatibility(enumA, enumD).isIncompatible)
   }
 
   /**
@@ -118,11 +118,11 @@ class EnumValueSerializerUpgradeTest extends TestLogger with JUnitSuiteLike {
   def checkDifferentIds(): Unit = {
     assertTrue(
       "Different ids should cause a migration.",
-      checkCompatibility(enumA, enumE).isRequiresMigration)
+      checkCompatibility(enumA, enumE).isIncompatible)
   }
 
   def checkCompatibility(enumSourceA: String, enumSourceB: String)
-    : CompatibilityResult[Enumeration#Value] = {
+    : TypeSerializerSchemaCompatibility[Enumeration#Value] = {
     import EnumValueSerializerUpgradeTest._
 
     val classLoader = compileAndLoadEnum(tempFolder.newFolder(), s"$enumName.scala", enumSourceA)
@@ -134,7 +134,7 @@ class EnumValueSerializerUpgradeTest extends TestLogger with JUnitSuiteLike {
 
     val baos = new ByteArrayOutputStream()
     val output = new DataOutputViewStreamWrapper(baos)
-    TypeSerializerSerializationUtil.writeSerializerConfigSnapshot(output, snapshot)
+    TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(output, snapshot)
 
     output.close()
     baos.close()
@@ -144,13 +144,14 @@ class EnumValueSerializerUpgradeTest extends TestLogger with JUnitSuiteLike {
 
     val classLoader2 = compileAndLoadEnum(tempFolder.newFolder(), s"$enumName.scala", enumSourceB)
 
-    val snapshot2 = TypeSerializerSerializationUtil.readSerializerConfigSnapshot(
+    val snapshot2 = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(
       input,
       classLoader2)
     val enum2 = instantiateEnum[Enumeration](classLoader2, enumName)
 
     val enumValueSerializer2 = new EnumValueSerializer(enum2)
-    enumValueSerializer2.ensureCompatibility(snapshot2)
+    snapshot2.resolveSchemaCompatibility(enumValueSerializer2)
+      .asInstanceOf[TypeSerializerSchemaCompatibility[Enumeration#Value]]
   }
 }
 
diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java
index 7483089106f..62f2843174d 100644
--- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java
+++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/AbstractRocksDBState.java
@@ -21,6 +21,7 @@
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos;
+import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
 import org.apache.flink.queryablestate.client.state.serialization.KvStateSerializer;
@@ -33,6 +34,8 @@
 import org.rocksdb.RocksDBException;
 import org.rocksdb.WriteOptions;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 
 /**
@@ -225,4 +228,24 @@ protected V getDefaultValue() {
 			return null;
 		}
 	}
+
+	public byte[] migrateSerializedValue(
+			byte[] serializedOldValue,
+			TypeSerializer<V> migrationValueSerializer,
+			TypeSerializer<V> valueSerializer) throws IOException {
+
+		V value = migrationValueSerializer.deserialize(new DataInputViewStreamWrapper(new ByteArrayInputStream(
+			serializedOldValue)));
+
+		ByteArrayOutputStream baos = new ByteArrayOutputStream();
+		DataOutputViewStreamWrapper out = new DataOutputViewStreamWrapper(baos);
+		valueSerializer.serialize(value, out);
+
+		byte[] result = baos.toByteArray();
+
+		out.close();
+		baos.close();
+
+		return result;
+	}
 }
diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
index e5f443a95ce..c66ced9236e 100644
--- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
+++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java
@@ -29,7 +29,7 @@
 import org.apache.flink.api.common.state.ValueStateDescriptor;
 import org.apache.flink.api.common.typeutils.CompatibilityUtil;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.ConfigConstants;
@@ -612,11 +612,9 @@ private void restoreKVStateMetaData() throws IOException, StateMigrationExceptio
 			// check for key serializer compatibility; this also reconfigures the
 			// key serializer to be compatible, if it is required and is possible
 			if (CompatibilityUtil.resolveCompatibilityResult(
-				serializationProxy.getKeySerializer(),
-				UnloadableDummyTypeSerializer.class,
 				serializationProxy.getKeySerializerConfigSnapshot(),
 				rocksDBKeyedStateBackend.keySerializer)
-				.isRequiresMigration()) {
+				.isIncompatible()) {
 
 				// TODO replace with state migration; note that key hash codes need to remain the same after migration
 				throw new StateMigrationException("The new key serializer is not compatible to read previous keys. " +
@@ -646,8 +644,8 @@ private void restoreKVStateMetaData() throws IOException, StateMigrationExceptio
 						new RegisteredKeyedBackendStateMetaInfo<>(
 							restoredMetaInfo.getStateType(),
 							restoredMetaInfo.getName(),
-							restoredMetaInfo.getNamespaceSerializer(),
-							restoredMetaInfo.getStateSerializer());
+							restoredMetaInfo.getNamespaceSerializerConfigSnapshot().restoreSerializer(),
+							restoredMetaInfo.getStateSerializerConfigSnapshot().restoreSerializer());
 
 					rocksDBKeyedStateBackend.restoredKvStateMetaInfos.put(restoredMetaInfo.getName(), restoredMetaInfo);
 
@@ -952,8 +950,8 @@ private ColumnFamilyHandle getOrRegisterColumnFamilyHandle(
 					new RegisteredKeyedBackendStateMetaInfo<>(
 						stateMetaInfoSnapshot.getStateType(),
 						stateMetaInfoSnapshot.getName(),
-						stateMetaInfoSnapshot.getNamespaceSerializer(),
-						stateMetaInfoSnapshot.getStateSerializer());
+						stateMetaInfoSnapshot.getNamespaceSerializerConfigSnapshot().restoreSerializer(),
+						stateMetaInfoSnapshot.getStateSerializerConfigSnapshot().restoreSerializer());
 
 				registeredStateMetaInfoEntry =
 					new Tuple2<>(
@@ -1085,8 +1083,8 @@ private void restoreLocalStateIntoFullInstance(
 					new RegisteredKeyedBackendStateMetaInfo<>(
 						stateMetaInfoSnapshot.getStateType(),
 						stateMetaInfoSnapshot.getName(),
-						stateMetaInfoSnapshot.getNamespaceSerializer(),
-						stateMetaInfoSnapshot.getStateSerializer());
+						stateMetaInfoSnapshot.getNamespaceSerializerConfigSnapshot().restoreSerializer(),
+						stateMetaInfoSnapshot.getStateSerializerConfigSnapshot().restoreSerializer());
 
 				stateBackend.kvStateInformation.put(
 					stateMetaInfoSnapshot.getName(),
@@ -1156,11 +1154,9 @@ private void restoreInstanceDirectoryFromPath(Path source) throws IOException {
 				// check for key serializer compatibility; this also reconfigures the
 				// key serializer to be compatible, if it is required and is possible
 				if (CompatibilityUtil.resolveCompatibilityResult(
-					serializationProxy.getKeySerializer(),
-					UnloadableDummyTypeSerializer.class,
 					serializationProxy.getKeySerializerConfigSnapshot(),
 					stateBackend.keySerializer)
-					.isRequiresMigration()) {
+					.isIncompatible()) {
 
 					// TODO replace with state migration; note that key hash codes need to remain the same after migration
 					throw new StateMigrationException("The new key serializer is not compatible to read previous keys. " +
@@ -1255,31 +1251,20 @@ private void copyStateDataHandleData(
 	 * already have a registered entry for that and return it (after some necessary state compatibility checks)
 	 * or create a new one if it does not exist.
 	 */
-	private <N, S> Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<N, S>> tryRegisterKvStateInformation(
-			StateDescriptor<?, S> stateDesc,
-			TypeSerializer<N> namespaceSerializer) throws StateMigrationException, IOException {
+	@SuppressWarnings({"unchecked", "rawtypes"})
+	private <N, S extends State, SV> Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<N, SV>> tryRegisterKvStateInformation(
+			StateDescriptor<S, SV> stateDesc,
+			TypeSerializer<N> namespaceSerializer) throws Exception {
 
 		Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<?, ?>> stateInfo =
 			kvStateInformation.get(stateDesc.getName());
 
-		RegisteredKeyedBackendStateMetaInfo<N, S> newMetaInfo;
+		RegisteredKeyedBackendStateMetaInfo<N, SV> newMetaInfo;
 		if (stateInfo != null) {
-
-			@SuppressWarnings("unchecked")
-			RegisteredKeyedBackendStateMetaInfo.Snapshot<N, S> restoredMetaInfoSnapshot =
-				(RegisteredKeyedBackendStateMetaInfo.Snapshot<N, S>) restoredKvStateMetaInfos.get(stateDesc.getName());
-
-			Preconditions.checkState(
-				restoredMetaInfoSnapshot != null,
-				"Requested to check compatibility of a restored RegisteredKeyedBackendStateMetaInfo," +
-					" but its corresponding restored snapshot cannot be found.");
-
-			newMetaInfo = RegisteredKeyedBackendStateMetaInfo.resolveKvStateCompatibility(
-				restoredMetaInfoSnapshot,
+			newMetaInfo = migrateStateIfNecessary(
+				stateDesc,
 				namespaceSerializer,
-				stateDesc);
-
-			stateInfo.f1 = newMetaInfo;
+				stateInfo);
 		} else {
 			String stateName = stateDesc.getName();
 
@@ -1298,6 +1283,130 @@ private void copyStateDataHandleData(
 		return Tuple2.of(stateInfo.f0, newMetaInfo);
 	}
 
+	private <N, S extends State, SV> RegisteredKeyedBackendStateMetaInfo<N, SV> migrateStateIfNecessary(
+			StateDescriptor<S, SV> stateDesc,
+			TypeSerializer<N> namespaceSerializer,
+			Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<?, ?>> stateInfo) throws Exception {
+
+		@SuppressWarnings("unchecked")
+		RegisteredKeyedBackendStateMetaInfo.Snapshot<N, SV> restoredMetaInfoSnapshot =
+			(RegisteredKeyedBackendStateMetaInfo.Snapshot<N, SV>) restoredKvStateMetaInfos.get(
+				stateDesc.getName());
+
+		Preconditions.checkState(
+			restoredMetaInfoSnapshot != null,
+			"Requested to check compatibility of a restored RegisteredKeyedBackendStateMetaInfo," +
+				" but its corresponding restored snapshot cannot be found.");
+
+		StateUtil.checkStateTypeCompatibility(restoredMetaInfoSnapshot, stateDesc);
+
+		TypeSerializer<SV> stateSerializer = stateDesc.getSerializer();
+
+		RegisteredKeyedBackendStateMetaInfo<N, SV> newMetaInfo = new RegisteredKeyedBackendStateMetaInfo<>(
+			stateDesc.getType(),
+			stateDesc.getName(),
+			namespaceSerializer,
+			stateSerializer);
+
+		// check compatibility results to determine if state migration is required
+		TypeSerializerSchemaCompatibility<N> namespaceCompatibility = CompatibilityUtil.resolveCompatibilityResult(
+			restoredMetaInfoSnapshot.getNamespaceSerializerConfigSnapshot(),
+			namespaceSerializer);
+
+		TypeSerializerSchemaCompatibility<SV> stateCompatibility = CompatibilityUtil.resolveCompatibilityResult(
+			restoredMetaInfoSnapshot.getStateSerializerConfigSnapshot(),
+			stateSerializer);
+
+		if (namespaceCompatibility.isIncompatible()) {
+			throw new UnsupportedOperationException(
+				"Changing the namespace TypeSerializer in an incompatible way is currently not supported.");
+		}
+
+		if (stateCompatibility.isIncompatible()) {
+			migrateStateValue(stateDesc, stateInfo, restoredMetaInfoSnapshot, newMetaInfo);
+		} else if (stateCompatibility.isCompatibleAfterReconfiguration()) {
+			// need to use the reconfigured serializer
+			newMetaInfo = new RegisteredKeyedBackendStateMetaInfo<>(
+				newMetaInfo.getStateType(),
+				newMetaInfo.getName(),
+				newMetaInfo.getNamespaceSerializer(),
+				stateCompatibility.getReconfiguredNewSerializer());
+		}
+
+		stateInfo.f1 = newMetaInfo;
+		return newMetaInfo;
+	}
+
+	/**
+	 * Migrate only the state value, that is the "value" that is stored in RocksDB. We don't migrate
+	 * the key here, which is made up of key group, key, namespace and map key
+	 * (in case of MapState).
+	 */
+	private <N, S extends State, SV> void migrateStateValue(
+			StateDescriptor<S, SV> stateDesc,
+			Tuple2<ColumnFamilyHandle, RegisteredKeyedBackendStateMetaInfo<?, ?>> stateInfo,
+			RegisteredKeyedBackendStateMetaInfo.Snapshot<N, SV> restoredMetaInfoSnapshot,
+			RegisteredKeyedBackendStateMetaInfo<N, SV> newMetaInfo) throws Exception {
+
+		if (stateDesc.getType().equals(StateDescriptor.Type.MAP)) {
+			throw new UnsupportedOperationException(
+				"Changing the TypeSerializers of a MapState in an incompatible way is currently not supported.");
+		}
+
+		LOG.info(
+			"Performing state migration for state {} because the state serializer changed in an incompatible way.",
+			stateDesc);
+
+		// we need to get an actual state instance because migration is different
+		// for different state types. For example, ListState needs to deal with
+		// individual elements
+		StateFactory stateFactory = STATE_FACTORIES.get(stateDesc.getClass());
+		if (stateFactory == null) {
+			String message = String.format("State %s is not supported by %s",
+				stateDesc.getClass(), this.getClass());
+			throw new FlinkRuntimeException(message);
+		}
+
+		State state = stateFactory.createState(
+			stateDesc,
+			Tuple2.of(stateInfo.f0, newMetaInfo),
+			RocksDBKeyedStateBackend.this);
+
+		if (!(state instanceof AbstractRocksDBState)) {
+			throw new FlinkRuntimeException(
+				"State should be an AbstractRocksDBState but is " + state);
+		}
+
+		@SuppressWarnings("unchecked")
+		AbstractRocksDBState<?, ?, SV, ?> rocksDBState = (AbstractRocksDBState<?, ?, SV, ?>) state;
+
+		Snapshot rocksDBSnapshot = db.getSnapshot();
+		try (ReadOptions readOptions = new ReadOptions().setSnapshot(rocksDBSnapshot);
+		     RocksIteratorWrapper iterator = getRocksIterator(db, stateInfo.f0, readOptions)) {
+
+			iterator.seekToFirst();
+
+			while (iterator.isValid()) {
+
+				byte[] serializedValue = iterator.value();
+
+				byte[] migratedSerializedValue = rocksDBState.migrateSerializedValue(
+					serializedValue,
+					restoredMetaInfoSnapshot.getStateSerializerConfigSnapshot().restoreSerializer(),
+					stateDesc.getSerializer());
+
+				db.put(stateInfo.f0, iterator.key(), migratedSerializedValue);
+
+				iterator.next();
+			}
+		} finally {
+			if (rocksDBSnapshot != null) {
+				db.releaseSnapshot(rocksDBSnapshot);
+				rocksDBSnapshot.close();
+			}
+		}
+	}
+
 	/**
 	 * Creates a column family handle for use with a k/v state.
 	 */
diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
index 03faa442737..35bbed3ce66 100644
--- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
+++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBListState.java
@@ -23,6 +23,7 @@
 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.api.common.typeutils.base.ListSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.memory.DataInputViewStreamWrapper;
 import org.apache.flink.core.memory.DataOutputViewStreamWrapper;
@@ -35,6 +36,7 @@
 import org.rocksdb.RocksDBException;
 
 import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
@@ -267,4 +269,41 @@ public void addAll(List<V> values) {
 			((ListStateDescriptor<E>) stateDesc).getElementSerializer(),
 			backend);
 	}
+
+	@Override
+	public byte[] migrateSerializedValue(
+		byte[] serializedOldValue,
+		TypeSerializer<List<V>> migrationListSerializer,
+		TypeSerializer<List<V>> listSerializer) throws IOException {
+
+		Preconditions.checkArgument(migrationListSerializer instanceof ListSerializer);
+		Preconditions.checkArgument(listSerializer instanceof ListSerializer);
+
+		TypeSerializer<V> migrationElementSerializer =
+			((ListSerializer<V>) migrationListSerializer).getElementSerializer();
+		TypeSerializer<V> elementSerializer =
+			((ListSerializer<V>) listSerializer).getElementSerializer();
+
+		ByteArrayInputStream bais = new ByteArrayInputStream(serializedOldValue);
+		DataInputViewStreamWrapper in = new DataInputViewStreamWrapper(bais);
+
+		ByteArrayOutputStream baos = new ByteArrayOutputStream();
+		DataOutputViewStreamWrapper out = new DataOutputViewStreamWrapper(baos);
+
+		while (in.available() > 0) {
+			V element = migrationElementSerializer.deserialize(in);
+			elementSerializer.serialize(element, out);
+			if (in.available() > 0) {
+				in.readByte();
+				out.write(DELIMITER);
+			}
+		}
+
+		byte[] result = baos.toByteArray();
+
+		out.close();
+		baos.close();
+
+		return result;
+	}
 }
diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendMigrationTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendMigrationTest.java
new file mode 100644
index 00000000000..8f46255c0fd
--- /dev/null
+++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendMigrationTest.java
@@ -0,0 +1,111 @@
+/*
+ * 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.contrib.streaming.state;
+
+import org.apache.flink.api.common.state.ValueState;
+import org.apache.flink.core.testutils.OneShotLatch;
+import org.apache.flink.runtime.state.StateBackendMigrationTestBase;
+import org.apache.flink.runtime.state.filesystem.FsStateBackend;
+import org.apache.flink.runtime.util.BlockerCheckpointStreamFactory;
+
+import org.apache.commons.io.IOUtils;
+import org.junit.After;
+import org.junit.Rule;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.rocksdb.RocksObject;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.verify;
+import static org.mockito.internal.verification.VerificationModeFactory.times;
+
+/**
+ * Tests for the partitioned state part of {@link RocksDBStateBackend}.
+ */
+@RunWith(Parameterized.class)
+public class RocksDBStateBackendMigrationTest extends StateBackendMigrationTestBase<RocksDBStateBackend> {
+
+	private OneShotLatch blocker;
+	private OneShotLatch waiter;
+	private BlockerCheckpointStreamFactory testStreamFactory;
+	private RocksDBKeyedStateBackend<Integer> keyedStateBackend;
+	private List<RocksObject> allCreatedCloseables;
+	private ValueState<Integer> testState1;
+	private ValueState<String> testState2;
+
+	@Parameterized.Parameters(name = "Incremental checkpointing: {0}")
+	public static Collection<Boolean> parameters() {
+		return Arrays.asList(false, true);
+	}
+
+	@Parameterized.Parameter
+	public boolean enableIncrementalCheckpointing;
+
+	@Rule
+	public final TemporaryFolder tempFolder = new TemporaryFolder();
+
+	// Store it because we need it for the cleanup test.
+	private String dbPath;
+
+	@Override
+	protected RocksDBStateBackend getStateBackend() throws IOException {
+		dbPath = tempFolder.newFolder().getAbsolutePath();
+		String checkpointPath = tempFolder.newFolder().toURI().toString();
+		RocksDBStateBackend backend = new RocksDBStateBackend(new FsStateBackend(checkpointPath), enableIncrementalCheckpointing);
+		backend.setDbStoragePath(dbPath);
+		return backend;
+	}
+
+	// small safety net for instance cleanups, so that no native objects are left
+	@After
+	public void cleanupRocksDB() {
+		if (keyedStateBackend != null) {
+			IOUtils.closeQuietly(keyedStateBackend);
+			keyedStateBackend.dispose();
+		}
+
+		if (allCreatedCloseables != null) {
+			for (RocksObject rocksCloseable : allCreatedCloseables) {
+				verify(rocksCloseable, times(1)).close();
+			}
+			allCreatedCloseables = null;
+		}
+	}
+
+	@Override
+	protected void verifyValueStateWithIncompatibleSerializerCounts() {
+		assertEquals((Integer) 2, CustomStringSerializer.serializeCalled.get(SerializerVersion.INITIAL));
+		assertEquals((Integer) 6, CustomStringSerializer.serializeCalled.get(SerializerVersion.NEW));
+		assertEquals((Integer) 5, CustomStringSerializer.deserializeCalled.get(SerializerVersion.NEW));
+		assertEquals((Integer) 2, CustomStringSerializer.deserializeCalled.get(SerializerVersion.RESTORE));
+	}
+
+	@Override
+	protected void verifyValueStateWithReconfiguredSerializerCounts() {
+		assertEquals((Integer) 2, CustomStringSerializer.serializeCalled.get(SerializerVersion.INITIAL));
+		assertEquals((Integer) 2, CustomStringSerializer.serializeCalled.get(SerializerVersion.RECONFIGURED));
+		assertEquals((Integer) 3, CustomStringSerializer.deserializeCalled.get(SerializerVersion.RECONFIGURED));
+	}
+}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
index c2ebdf483ed..fcb6a828f93 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/CoGroupedStreams.java
@@ -24,16 +24,12 @@
 import org.apache.flink.api.common.functions.CoGroupFunction;
 import org.apache.flink.api.common.functions.MapFunction;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializer;
 import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
 import org.apache.flink.api.java.operators.translation.WrappingFunction;
-import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.TypeExtractor;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -43,6 +39,7 @@
 import org.apache.flink.streaming.api.windowing.triggers.Trigger;
 import org.apache.flink.streaming.api.windowing.windows.Window;
 import org.apache.flink.util.Collector;
+import org.apache.flink.util.Preconditions;
 
 import java.io.IOException;
 import java.util.ArrayList;
@@ -439,13 +436,21 @@ public boolean canEqual(Object obj) {
 		}
 	}
 
-	private static class UnionSerializer<T1, T2> extends TypeSerializer<TaggedUnion<T1, T2>> {
+	private static class UnionSerializer<T1, T2> extends CompositeTypeSerializer<TaggedUnion<T1, T2>> {
 		private static final long serialVersionUID = 1L;
 
 		private final TypeSerializer<T1> oneSerializer;
 		private final TypeSerializer<T2> twoSerializer;
 
 		public UnionSerializer(TypeSerializer<T1> oneSerializer, TypeSerializer<T2> twoSerializer) {
+
+			super(
+				new UnionSerializerConfigSnapshot<>(
+					Preconditions.checkNotNull(oneSerializer),
+					Preconditions.checkNotNull(twoSerializer)),
+				oneSerializer,
+				twoSerializer);
+
 			this.oneSerializer = oneSerializer;
 			this.twoSerializer = twoSerializer;
 		}
@@ -551,50 +556,15 @@ public boolean equals(Object obj) {
 		public boolean canEqual(Object obj) {
 			return obj instanceof UnionSerializer;
 		}
-
-		@Override
-		public TypeSerializerConfigSnapshot snapshotConfiguration() {
-			return new UnionSerializerConfigSnapshot<>(oneSerializer, twoSerializer);
-		}
-
-		@Override
-		public CompatibilityResult<TaggedUnion<T1, T2>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-			if (configSnapshot instanceof UnionSerializerConfigSnapshot) {
-				List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> previousSerializersAndConfigs =
-					((UnionSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
-
-				CompatibilityResult<T1> oneSerializerCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-					previousSerializersAndConfigs.get(0).f0,
-					UnloadableDummyTypeSerializer.class,
-					previousSerializersAndConfigs.get(0).f1,
-					oneSerializer);
-
-				CompatibilityResult<T2> twoSerializerCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-					previousSerializersAndConfigs.get(1).f0,
-					UnloadableDummyTypeSerializer.class,
-					previousSerializersAndConfigs.get(1).f1,
-					twoSerializer);
-
-				if (!oneSerializerCompatResult.isRequiresMigration() && !twoSerializerCompatResult.isRequiresMigration()) {
-					return CompatibilityResult.compatible();
-				} else if (oneSerializerCompatResult.getConvertDeserializer() != null && twoSerializerCompatResult.getConvertDeserializer() != null) {
-					return CompatibilityResult.requiresMigration(
-						new UnionSerializer<>(
-							new TypeDeserializerAdapter<>(oneSerializerCompatResult.getConvertDeserializer()),
-							new TypeDeserializerAdapter<>(twoSerializerCompatResult.getConvertDeserializer())));
-				}
-			}
-
-			return CompatibilityResult.requiresMigration();
-		}
 	}
 
 	/**
 	 * The {@link TypeSerializerConfigSnapshot} for the {@link UnionSerializer}.
 	 */
-	public static class UnionSerializerConfigSnapshot<T1, T2> extends CompositeTypeSerializerConfigSnapshot {
+	public static class UnionSerializerConfigSnapshot<T1, T2>
+			extends CompositeTypeSerializerConfigSnapshot<TaggedUnion<T1, T2>> {
 
-		private static final int VERSION = 1;
+		private static final int VERSION = 2;
 
 		/** This empty nullary constructor is required for deserializing the configuration. */
 		public UnionSerializerConfigSnapshot() {}
@@ -607,6 +577,29 @@ public UnionSerializerConfigSnapshot(TypeSerializer<T1> oneSerializer, TypeSeria
 		public int getVersion() {
 			return VERSION;
 		}
+
+		@Override
+		public int[] getCompatibleVersions() {
+			return new int[]{VERSION, 1};
+		}
+
+		@Override
+		protected boolean containsSerializers() {
+			return getReadVersion() < 2;
+		}
+
+		@SuppressWarnings("unchecked")
+		@Override
+		protected TypeSerializer<TaggedUnion<T1, T2>> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+			return new UnionSerializer<>(
+				(TypeSerializer<T1>) restoredNestedSerializers[0],
+				(TypeSerializer<T2>) restoredNestedSerializers[1]);
+		}
+
+		@Override
+		protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+			return newSerializer instanceof UnionSerializer;
+		}
 	}
 
 	// ------------------------------------------------------------------------
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
index 2ffb6d5810e..3f8d4947a5d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.java
@@ -24,14 +24,10 @@
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.typeinfo.TypeHint;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializer;
 import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.runtime.state.CheckpointListener;
@@ -589,7 +585,7 @@ public String toString() {
 	 */
 	@VisibleForTesting
 	@Internal
-	public static final class StateSerializer<TXN, CONTEXT> extends TypeSerializer<State<TXN, CONTEXT>> {
+	public static final class StateSerializer<TXN, CONTEXT> extends CompositeTypeSerializer<State<TXN, CONTEXT>> {
 
 		private static final long serialVersionUID = 1L;
 
@@ -599,8 +595,16 @@ public String toString() {
 		public StateSerializer(
 				TypeSerializer<TXN> transactionSerializer,
 				TypeSerializer<CONTEXT> contextSerializer) {
-			this.transactionSerializer = checkNotNull(transactionSerializer);
-			this.contextSerializer = checkNotNull(contextSerializer);
+
+			super(
+				new StateSerializerConfigSnapshot<>(
+					checkNotNull(transactionSerializer),
+					checkNotNull(contextSerializer)),
+				transactionSerializer,
+				contextSerializer);
+
+			this.transactionSerializer = transactionSerializer;
+			this.contextSerializer = contextSerializer;
 		}
 
 		@Override
@@ -761,45 +765,6 @@ public int hashCode() {
 			result = 31 * result + contextSerializer.hashCode();
 			return result;
 		}
-
-		@Override
-		public TypeSerializerConfigSnapshot snapshotConfiguration() {
-			return new StateSerializerConfigSnapshot<>(transactionSerializer, contextSerializer);
-		}
-
-		@Override
-		public CompatibilityResult<State<TXN, CONTEXT>> ensureCompatibility(
-				TypeSerializerConfigSnapshot configSnapshot) {
-			if (configSnapshot instanceof StateSerializerConfigSnapshot) {
-				List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> previousSerializersAndConfigs =
-						((StateSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs();
-
-				CompatibilityResult<TXN> txnCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-						previousSerializersAndConfigs.get(0).f0,
-						UnloadableDummyTypeSerializer.class,
-						previousSerializersAndConfigs.get(0).f1,
-						transactionSerializer);
-
-				CompatibilityResult<CONTEXT> contextCompatResult = CompatibilityUtil.resolveCompatibilityResult(
-						previousSerializersAndConfigs.get(1).f0,
-						UnloadableDummyTypeSerializer.class,
-						previousSerializersAndConfigs.get(1).f1,
-						contextSerializer);
-
-				if (!txnCompatResult.isRequiresMigration() && !contextCompatResult.isRequiresMigration()) {
-					return CompatibilityResult.compatible();
-				} else {
-					if (txnCompatResult.getConvertDeserializer() != null && contextCompatResult.getConvertDeserializer() != null) {
-						return CompatibilityResult.requiresMigration(
-								new StateSerializer<>(
-										new TypeDeserializerAdapter<>(txnCompatResult.getConvertDeserializer()),
-										new TypeDeserializerAdapter<>(contextCompatResult.getConvertDeserializer())));
-					}
-				}
-			}
-
-			return CompatibilityResult.requiresMigration();
-		}
 	}
 
 	/**
@@ -809,9 +774,9 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() {
 	 */
 	@Internal
 	public static final class StateSerializerConfigSnapshot<TXN, CONTEXT>
-			extends CompositeTypeSerializerConfigSnapshot {
+			extends CompositeTypeSerializerConfigSnapshot<State<TXN, CONTEXT>> {
 
-		private static final int VERSION = 1;
+		private static final int VERSION = 2;
 
 		/** This empty nullary constructor is required for deserializing the configuration. */
 		public StateSerializerConfigSnapshot() {}
@@ -826,5 +791,28 @@ public StateSerializerConfigSnapshot(
 		public int getVersion() {
 			return VERSION;
 		}
+
+		@Override
+		public int[] getCompatibleVersions() {
+			return new int[]{VERSION, 1};
+		}
+
+		@Override
+		protected boolean containsSerializers() {
+			return getReadVersion() < 2;
+		}
+
+		@SuppressWarnings("unchecked")
+		@Override
+		protected TypeSerializer<State<TXN, CONTEXT>> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+			return new StateSerializer<>(
+				(TypeSerializer<TXN>) restoredNestedSerializers[0],
+				(TypeSerializer<CONTEXT>) restoredNestedSerializers[1]);
+		}
+
+		@Override
+		protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+			return newSerializer instanceof StateSerializer;
+		}
 	}
 }
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/HeapInternalTimerService.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/HeapInternalTimerService.java
index c5a68fb912a..8f18ea8955a 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/HeapInternalTimerService.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/HeapInternalTimerService.java
@@ -19,9 +19,9 @@
 package org.apache.flink.streaming.api.operators;
 
 import org.apache.flink.annotation.VisibleForTesting;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
 import org.apache.flink.api.common.typeutils.CompatibilityUtil;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback;
 import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService;
@@ -136,19 +136,15 @@ public void startTimerService(
 
 			// the following is the case where we restore
 			if (restoredTimersSnapshot != null) {
-				CompatibilityResult<K> keySerializerCompatibility = CompatibilityUtil.resolveCompatibilityResult(
-					this.keyDeserializer,
-					null,
+				TypeSerializerSchemaCompatibility<K> keySerializerCompatibility = CompatibilityUtil.resolveCompatibilityResult(
 					restoredTimersSnapshot.getKeySerializerConfigSnapshot(),
 					keySerializer);
 
-				CompatibilityResult<N> namespaceSerializerCompatibility = CompatibilityUtil.resolveCompatibilityResult(
-					this.namespaceDeserializer,
-					null,
+				TypeSerializerSchemaCompatibility<N> namespaceSerializerCompatibility = CompatibilityUtil.resolveCompatibilityResult(
 					restoredTimersSnapshot.getNamespaceSerializerConfigSnapshot(),
 					namespaceSerializer);
 
-				if (keySerializerCompatibility.isRequiresMigration() || namespaceSerializerCompatibility.isRequiresMigration()) {
+				if (keySerializerCompatibility.isIncompatible() || namespaceSerializerCompatibility.isIncompatible()) {
 					throw new IllegalStateException("Tried to initialize restored TimerService " +
 						"with incompatible serializers than those used to snapshot its state.");
 				}
@@ -257,9 +253,7 @@ public void advanceWatermark(long time) throws Exception {
 	public InternalTimersSnapshot<K, N> snapshotTimersForKeyGroup(int keyGroupIdx) {
 		return new InternalTimersSnapshot<>(
 				keySerializer,
-				keySerializer.snapshotConfiguration(),
 				namespaceSerializer,
-				namespaceSerializer.snapshotConfiguration(),
 				eventTimeTimersQueue.getTimersForKeyGroup(keyGroupIdx),
 				processingTimeTimersQueue.getTimersForKeyGroup(keyGroupIdx));
 	}
@@ -275,13 +269,18 @@ public void advanceWatermark(long time) throws Exception {
 	public void restoreTimersForKeyGroup(InternalTimersSnapshot<?, ?> restoredSnapshot, int keyGroupIdx) {
 		this.restoredTimersSnapshot = (InternalTimersSnapshot<K, N>) restoredSnapshot;
 
-		if (areSnapshotSerializersIncompatible(restoredSnapshot)) {
+		TypeSerializer<K> restoredKeyDeserializer =
+			restoredTimersSnapshot.getKeySerializerConfigSnapshot().restoreSerializer();
+		TypeSerializer<N> restoredNamespaceDeserializer =
+			restoredTimersSnapshot.getNamespaceSerializerConfigSnapshot().restoreSerializer();
+
+		if (areSnapshotSerializersIncompatible(restoredKeyDeserializer, restoredNamespaceDeserializer)) {
 			throw new IllegalArgumentException("Tried to restore timers " +
 				"for the same service with different serializers.");
 		}
 
-		this.keyDeserializer = restoredTimersSnapshot.getKeySerializer();
-		this.namespaceDeserializer = restoredTimersSnapshot.getNamespaceSerializer();
+		this.keyDeserializer = restoredKeyDeserializer;
+		this.namespaceDeserializer = restoredNamespaceDeserializer;
 
 		checkArgument(localKeyGroupRange.contains(keyGroupIdx),
 			"Key Group " + keyGroupIdx + " does not belong to the local range.");
@@ -336,8 +335,11 @@ int getLocalKeyGroupRangeStartIdx() {
 		return processingTimeTimersQueue.getTimersByKeyGroup();
 	}
 
-	private boolean areSnapshotSerializersIncompatible(InternalTimersSnapshot<?, ?> restoredSnapshot) {
-		return (this.keyDeserializer != null && !this.keyDeserializer.equals(restoredSnapshot.getKeySerializer())) ||
-			(this.namespaceDeserializer != null && !this.namespaceDeserializer.equals(restoredSnapshot.getNamespaceSerializer()));
+	private boolean areSnapshotSerializersIncompatible(
+			TypeSerializer<K> restoredKeySerializer,
+			TypeSerializer<N> restoredNamespaceSerializer) {
+
+		return (this.keyDeserializer != null && !this.keyDeserializer.equals(restoredKeySerializer)) ||
+			(this.namespaceDeserializer != null && !this.namespaceDeserializer.equals(restoredNamespaceSerializer));
 	}
 }
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceSerializationProxy.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceSerializationProxy.java
index efa93d3e266..900a81cc0d2 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceSerializationProxy.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimerServiceSerializationProxy.java
@@ -36,7 +36,7 @@
 @Internal
 public class InternalTimerServiceSerializationProxy<K> extends PostVersionedIOReadableWritable {
 
-	public static final int VERSION = 1;
+	public static final int VERSION = 2;
 
 	/** The key-group timer services to write / read. */
 	private Map<String, HeapInternalTimerService<K, ?>> timerServices;
@@ -88,6 +88,11 @@ public int getVersion() {
 		return VERSION;
 	}
 
+	@Override
+	public int[] getCompatibleVersions() {
+		return new int[]{VERSION, 1};
+	}
+
 	@Override
 	public void write(DataOutputView out) throws IOException {
 		super.write(out);
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimersSnapshot.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimersSnapshot.java
index ddaee6b2e7a..cb2e0a16323 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimersSnapshot.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimersSnapshot.java
@@ -32,10 +32,8 @@
  */
 public class InternalTimersSnapshot<K, N> {
 
-	private TypeSerializer<K> keySerializer;
-	private TypeSerializerConfigSnapshot keySerializerConfigSnapshot;
-	private TypeSerializer<N> namespaceSerializer;
-	private TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot;
+	private TypeSerializerConfigSnapshot<K> keySerializerConfigSnapshot;
+	private TypeSerializerConfigSnapshot<N> namespaceSerializerConfigSnapshot;
 
 	private Set<InternalTimer<K, N>> eventTimeTimers;
 	private Set<InternalTimer<K, N>> processingTimeTimers;
@@ -46,49 +44,31 @@ public InternalTimersSnapshot() {}
 	/** Constructor to use when snapshotting the timers. */
 	public InternalTimersSnapshot(
 			TypeSerializer<K> keySerializer,
-			TypeSerializerConfigSnapshot keySerializerConfigSnapshot,
 			TypeSerializer<N> namespaceSerializer,
-			TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot,
 			@Nullable Set<InternalTimer<K, N>> eventTimeTimers,
 			@Nullable Set<InternalTimer<K, N>> processingTimeTimers) {
 
-		this.keySerializer = Preconditions.checkNotNull(keySerializer);
-		this.keySerializerConfigSnapshot = Preconditions.checkNotNull(keySerializerConfigSnapshot);
-		this.namespaceSerializer = Preconditions.checkNotNull(namespaceSerializer);
-		this.namespaceSerializerConfigSnapshot = Preconditions.checkNotNull(namespaceSerializerConfigSnapshot);
+		Preconditions.checkNotNull(keySerializer);
+		this.keySerializerConfigSnapshot = Preconditions.checkNotNull(keySerializer.snapshotConfiguration());
+		Preconditions.checkNotNull(namespaceSerializer);
+		this.namespaceSerializerConfigSnapshot = Preconditions.checkNotNull(namespaceSerializer.snapshotConfiguration());
 		this.eventTimeTimers = eventTimeTimers;
 		this.processingTimeTimers = processingTimeTimers;
 	}
 
-	public TypeSerializer<K> getKeySerializer() {
-		return keySerializer;
-	}
-
-	public void setKeySerializer(TypeSerializer<K> keySerializer) {
-		this.keySerializer = keySerializer;
-	}
-
 	public TypeSerializerConfigSnapshot getKeySerializerConfigSnapshot() {
 		return keySerializerConfigSnapshot;
 	}
 
-	public void setKeySerializerConfigSnapshot(TypeSerializerConfigSnapshot keySerializerConfigSnapshot) {
+	public void setKeySerializerConfigSnapshot(TypeSerializerConfigSnapshot<K> keySerializerConfigSnapshot) {
 		this.keySerializerConfigSnapshot = keySerializerConfigSnapshot;
 	}
 
-	public TypeSerializer<N> getNamespaceSerializer() {
-		return namespaceSerializer;
-	}
-
-	public void setNamespaceSerializer(TypeSerializer<N> namespaceSerializer) {
-		this.namespaceSerializer = namespaceSerializer;
-	}
-
 	public TypeSerializerConfigSnapshot getNamespaceSerializerConfigSnapshot() {
 		return namespaceSerializerConfigSnapshot;
 	}
 
-	public void setNamespaceSerializerConfigSnapshot(TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot) {
+	public void setNamespaceSerializerConfigSnapshot(TypeSerializerConfigSnapshot<N> namespaceSerializerConfigSnapshot) {
 		this.namespaceSerializerConfigSnapshot = namespaceSerializerConfigSnapshot;
 	}
 
@@ -108,6 +88,17 @@ public void setProcessingTimeTimers(Set<InternalTimer<K, N>> processingTimeTimer
 		this.processingTimeTimers = processingTimeTimers;
 	}
 
+	public TimerHeapInternalTimer.TimerSerializer<K, N> createTimerSerializer() {
+		Preconditions.checkState(keySerializerConfigSnapshot != null && namespaceSerializerConfigSnapshot != null,
+			"Key / namespace serializer config snapshots are null; if the timer snapshot" +
+				" was restored from a checkpoint, the serializer config snapshots must be restored first before" +
+				" attempting to create the timer serializer.");
+
+		return new TimerHeapInternalTimer.TimerSerializer<>(
+			keySerializerConfigSnapshot.restoreSerializer(),
+			namespaceSerializerConfigSnapshot.restoreSerializer());
+	}
+
 	@Override
 	public boolean equals(Object obj) {
 		return super.equals(obj);
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimersSnapshotReaderWriters.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimersSnapshotReaderWriters.java
index 05b77a7143c..b3c1c3359ce 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimersSnapshotReaderWriters.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimersSnapshotReaderWriters.java
@@ -19,8 +19,10 @@
 package org.apache.flink.streaming.api.operators;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.common.typeutils.BackwardsCompatibleConfigSnapshot;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshotSerializationUtil;
 import org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream;
@@ -49,7 +51,8 @@
 	// -------------------------------------------------------------------------------
 	//  Writers
 	//   - pre-versioned: Flink 1.4.0
-	//   - v1: Flink 1.4.1
+	//   - v1: Flink 1.4.1, 1.5.x
+	//   - v2: Flink 1.6.x
 	// -------------------------------------------------------------------------------
 
 	public static <K, N> InternalTimersSnapshotWriter getWriterForVersion(int version, InternalTimersSnapshot<K, N> timersSnapshot) {
@@ -58,9 +61,13 @@
 			case NO_VERSION:
 				return new InternalTimersSnapshotWriterPreVersioned<>(timersSnapshot);
 
-			case InternalTimerServiceSerializationProxy.VERSION:
+			case 1:
 				return new InternalTimersSnapshotWriterV1<>(timersSnapshot);
 
+			// current version:
+			case InternalTimerServiceSerializationProxy.VERSION:
+				return new InternalTimersSnapshotWriterV2<>(timersSnapshot);
+
 			default:
 				// guard for future
 				throw new IllegalStateException(
@@ -96,9 +103,7 @@ public AbstractInternalTimersSnapshotWriter(InternalTimersSnapshot<K, N> timersS
 		public final void writeTimersSnapshot(DataOutputView out) throws IOException {
 			writeKeyAndNamespaceSerializers(out);
 
-			TimerHeapInternalTimer.TimerSerializer<K, N> timerSerializer = new TimerHeapInternalTimer.TimerSerializer<>(
-				timersSnapshot.getKeySerializer(),
-				timersSnapshot.getNamespaceSerializer());
+			TimerHeapInternalTimer.TimerSerializer<K, N> timerSerializer = timersSnapshot.createTimerSerializer();
 
 			// write the event time timers
 			Set<InternalTimer<K, N>> eventTimers = timersSnapshot.getEventTimeTimers();
@@ -134,8 +139,10 @@ public InternalTimersSnapshotWriterPreVersioned(InternalTimersSnapshot<K, N> tim
 		protected void writeKeyAndNamespaceSerializers(DataOutputView out) throws IOException {
 			// the pre-versioned format only serializes the serializers, without their configuration snapshots
 			try (ByteArrayOutputStreamWithPos stream = new ByteArrayOutputStreamWithPos()) {
-				InstantiationUtil.serializeObject(stream, timersSnapshot.getKeySerializer());
-				InstantiationUtil.serializeObject(stream, timersSnapshot.getNamespaceSerializer());
+				// state meta info snapshots no longer contain serializers, so we use null just as a placeholder;
+				// this is maintained here to keep track of previous versions' serialization formats
+				InstantiationUtil.serializeObject(stream, null);
+				InstantiationUtil.serializeObject(stream, null);
 
 				out.write(stream.getBuf(), 0, stream.getPosition());
 			}
@@ -153,16 +160,34 @@ protected void writeKeyAndNamespaceSerializers(DataOutputView out) throws IOExce
 			// write key / namespace serializers, and their configuration snapshots
 			TypeSerializerSerializationUtil.writeSerializersAndConfigsWithResilience(
 				out,
+				// state meta info snapshots no longer contain serializers, so we use null just as a placeholder;
+				// this is maintained here to keep track of previous versions' serialization formats
 				Arrays.asList(
-					Tuple2.of(timersSnapshot.getKeySerializer(), timersSnapshot.getKeySerializerConfigSnapshot()),
-					Tuple2.of(timersSnapshot.getNamespaceSerializer(), timersSnapshot.getNamespaceSerializerConfigSnapshot())));
+					Tuple2.of(null, timersSnapshot.getKeySerializerConfigSnapshot()),
+					Tuple2.of(null, timersSnapshot.getNamespaceSerializerConfigSnapshot())));
+		}
+	}
+
+	private static class InternalTimersSnapshotWriterV2<K, N> extends AbstractInternalTimersSnapshotWriter<K, N> {
+
+		public InternalTimersSnapshotWriterV2(InternalTimersSnapshot<K, N> timersSnapshot) {
+			super(timersSnapshot);
+		}
+
+		@Override
+		protected void writeKeyAndNamespaceSerializers(DataOutputView out) throws IOException {
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(
+				out, timersSnapshot.getKeySerializerConfigSnapshot());
+			TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot(
+				out, timersSnapshot.getNamespaceSerializerConfigSnapshot());
 		}
 	}
 
 	// -------------------------------------------------------------------------------
 	//  Readers
 	//   - pre-versioned: Flink 1.4.0
-	//   - v1: Flink 1.4.1
+	//   - v1: Flink 1.4.1, 1.5.x
+	//   - v2: Flink 1.6.x
 	// -------------------------------------------------------------------------------
 
 	public static <K, N> InternalTimersSnapshotReader<K, N> getReaderForVersion(
@@ -172,9 +197,13 @@ protected void writeKeyAndNamespaceSerializers(DataOutputView out) throws IOExce
 			case NO_VERSION:
 				return new InternalTimersSnapshotReaderPreVersioned<>(userCodeClassLoader);
 
-			case InternalTimerServiceSerializationProxy.VERSION:
+			case 1:
 				return new InternalTimersSnapshotReaderV1<>(userCodeClassLoader);
 
+			// current version
+			case InternalTimerServiceSerializationProxy.VERSION:
+				return new InternalTimersSnapshotReaderV2<>(userCodeClassLoader);
+
 			default:
 				// guard for future
 				throw new IllegalStateException(
@@ -215,10 +244,7 @@ protected abstract void restoreKeyAndNamespaceSerializers(
 
 			restoreKeyAndNamespaceSerializers(restoredTimersSnapshot, in);
 
-			TimerHeapInternalTimer.TimerSerializer<K, N> timerSerializer =
-				new TimerHeapInternalTimer.TimerSerializer<>(
-					restoredTimersSnapshot.getKeySerializer(),
-					restoredTimersSnapshot.getNamespaceSerializer());
+			TimerHeapInternalTimer.TimerSerializer<K, N> timerSerializer = restoredTimersSnapshot.createTimerSerializer();
 
 			// read the event time timers
 			int sizeOfEventTimeTimers = in.readInt();
@@ -260,8 +286,14 @@ protected void restoreKeyAndNamespaceSerializers(
 
 			DataInputViewStream dis = new DataInputViewStream(in);
 			try {
-				restoredTimersSnapshot.setKeySerializer(InstantiationUtil.deserializeObject(dis, userCodeClassLoader, true));
-				restoredTimersSnapshot.setNamespaceSerializer(InstantiationUtil.deserializeObject(dis, userCodeClassLoader, true));
+				// Flink 1.4.0 and below did not write the serializer config snapshots for timers;
+				// we deserialize the written serializers, and then simply take a snapshot of them now
+
+				TypeSerializer<K> keySerializer = InstantiationUtil.deserializeObject(dis, userCodeClassLoader, true);
+				restoredTimersSnapshot.setKeySerializerConfigSnapshot(keySerializer.snapshotConfiguration());
+
+				TypeSerializer<N> namespaceSerializer = InstantiationUtil.deserializeObject(dis, userCodeClassLoader, true);
+				restoredTimersSnapshot.setNamespaceSerializerConfigSnapshot(namespaceSerializer.snapshotConfiguration());
 			} catch (ClassNotFoundException exception) {
 				throw new IOException(exception);
 			}
@@ -283,10 +315,31 @@ protected void restoreKeyAndNamespaceSerializers(
 			List<Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot>> serializersAndConfigs =
 				TypeSerializerSerializationUtil.readSerializersAndConfigsWithResilience(in, userCodeClassLoader);
 
-			restoredTimersSnapshot.setKeySerializer((TypeSerializer<K>) serializersAndConfigs.get(0).f0);
-			restoredTimersSnapshot.setKeySerializerConfigSnapshot(serializersAndConfigs.get(0).f1);
-			restoredTimersSnapshot.setNamespaceSerializer((TypeSerializer<N>) serializersAndConfigs.get(1).f0);
-			restoredTimersSnapshot.setNamespaceSerializerConfigSnapshot(serializersAndConfigs.get(1).f1);
+			restoredTimersSnapshot.setKeySerializerConfigSnapshot(
+				new BackwardsCompatibleConfigSnapshot(serializersAndConfigs.get(0).f1, serializersAndConfigs.get(0).f0));
+
+			restoredTimersSnapshot.setNamespaceSerializerConfigSnapshot(
+				new BackwardsCompatibleConfigSnapshot(serializersAndConfigs.get(1).f1, serializersAndConfigs.get(1).f0));
+		}
+	}
+
+	private static class InternalTimersSnapshotReaderV2<K, N> extends AbstractInternalTimersSnapshotReader<K, N> {
+
+		public InternalTimersSnapshotReaderV2(ClassLoader userCodeClassLoader) {
+			super(userCodeClassLoader);
+		}
+
+		@SuppressWarnings("unchecked")
+		@Override
+		protected void restoreKeyAndNamespaceSerializers(
+				InternalTimersSnapshot<K, N> restoredTimersSnapshot,
+				DataInputView in) throws IOException {
+
+			restoredTimersSnapshot.setKeySerializerConfigSnapshot(
+				TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(in, userCodeClassLoader));
+
+			restoredTimersSnapshot.setNamespaceSerializerConfigSnapshot(
+				TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(in, userCodeClassLoader));
 		}
 	}
 }
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java
index 906b0908abb..5f803f5c95e 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TimerHeapInternalTimer.java
@@ -19,9 +19,9 @@
 package org.apache.flink.streaming.api.operators;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility;
 import org.apache.flink.api.common.typeutils.base.LongSerializer;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
@@ -242,12 +242,12 @@ public int hashCode() {
 		}
 
 		@Override
-		public TypeSerializerConfigSnapshot snapshotConfiguration() {
+		public TypeSerializerConfigSnapshot<InternalTimer<K, N>> snapshotConfiguration() {
 			throw new UnsupportedOperationException("This serializer is not registered for managed state.");
 		}
 
 		@Override
-		public CompatibilityResult<InternalTimer<K, N>> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
+		public TypeSerializerSchemaCompatibility<InternalTimer<K, N>> ensureCompatibility(TypeSerializerConfigSnapshot<?> configSnapshot) {
 			throw new UnsupportedOperationException("This serializer is not registered for managed state.");
 		}
 	}
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java
index ba92416d792..dac295aa97d 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/streamrecord/StreamElementSerializer.java
@@ -19,24 +19,18 @@
 package org.apache.flink.streaming.runtime.streamrecord;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.api.common.typeutils.CompatibilityResult;
-import org.apache.flink.api.common.typeutils.CompatibilityUtil;
+import org.apache.flink.api.common.typeutils.CompositeTypeSerializer;
 import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.TypeDeserializerAdapter;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot;
-import org.apache.flink.api.common.typeutils.UnloadableDummyTypeSerializer;
-import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.core.memory.DataInputView;
 import org.apache.flink.core.memory.DataOutputView;
 import org.apache.flink.runtime.jobgraph.OperatorID;
 import org.apache.flink.streaming.api.watermark.Watermark;
 import org.apache.flink.streaming.runtime.streamstatus.StreamStatus;
+import org.apache.flink.util.Preconditions;
 
 import java.io.IOException;
 
-import static java.util.Objects.requireNonNull;
-
 /**
  * Serializer for {@link StreamRecord}, {@link Watermark}, {@link LatencyMarker}, and
  * {@link StreamStatus}.
@@ -47,7 +41,7 @@
  * @param <T> The type of value in the StreamRecord
  */
 @Internal
-public final class StreamElementSerializer<T> extends TypeSerializer<StreamElement> {
+public final class StreamElementSerializer<T> extends CompositeTypeSerializer<StreamElement> {
 
 	private static final long serialVersionUID = 1L;
 
@@ -61,10 +55,13 @@
 	private final TypeSerializer<T> typeSerializer;
 
 	public StreamElementSerializer(TypeSerializer<T> serializer) {
+
+		super(new StreamElementSerializerConfigSnapshot<>(Preconditions.checkNotNull(serializer)), serializer);
+
 		if (serializer instanceof StreamElementSerializer) {
 			throw new RuntimeException("StreamRecordSerializer given to StreamRecordSerializer as value TypeSerializer: " + serializer);
 		}
-		this.typeSerializer = requireNonNull(serializer);
+		this.typeSerializer = serializer;
 	}
 
 	public TypeSerializer<T> getContainedTypeSerializer() {
@@ -272,54 +269,12 @@ public int hashCode() {
 		return typeSerializer.hashCode();
 	}
 
-	// --------------------------------------------------------------------------------------------
-	// Serializer configuration snapshotting & compatibility
-	//
-	// This serializer may be used by Flink internal operators that need to checkpoint
-	// buffered records. Therefore, it may be part of managed state and need to implement
-	// the configuration snapshot and compatibility methods.
-	// --------------------------------------------------------------------------------------------
-
-	@Override
-	public StreamElementSerializerConfigSnapshot snapshotConfiguration() {
-		return new StreamElementSerializerConfigSnapshot<>(typeSerializer);
-	}
-
-	@Override
-	public CompatibilityResult<StreamElement> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) {
-		Tuple2<TypeSerializer<?>, TypeSerializerConfigSnapshot> previousTypeSerializerAndConfig;
-
-		// we are compatible for data written by ourselves or the legacy MultiplexingStreamRecordSerializer
-		if (configSnapshot instanceof StreamElementSerializerConfigSnapshot) {
-			previousTypeSerializerAndConfig =
-				((StreamElementSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig();
-		} else {
-			return CompatibilityResult.requiresMigration();
-		}
-
-		CompatibilityResult<T> compatResult = CompatibilityUtil.resolveCompatibilityResult(
-				previousTypeSerializerAndConfig.f0,
-				UnloadableDummyTypeSerializer.class,
-				previousTypeSerializerAndConfig.f1,
-				typeSerializer);
-
-		if (!compatResult.isRequiresMigration()) {
-			return CompatibilityResult.compatible();
-		} else if (compatResult.getConvertDeserializer() != null) {
-			return CompatibilityResult.requiresMigration(
-				new StreamElementSerializer<>(
-					new TypeDeserializerAdapter<>(compatResult.getConvertDeserializer())));
-		} else {
-			return CompatibilityResult.requiresMigration();
-		}
-	}
-
 	/**
 	 * Configuration snapshot specific to the {@link StreamElementSerializer}.
 	 */
-	public static final class StreamElementSerializerConfigSnapshot<T> extends CompositeTypeSerializerConfigSnapshot {
+	public static final class StreamElementSerializerConfigSnapshot<T> extends CompositeTypeSerializerConfigSnapshot<StreamElement> {
 
-		private static final int VERSION = 1;
+		private static final int VERSION = 2;
 
 		/** This empty nullary constructor is required for deserializing the configuration. */
 		public StreamElementSerializerConfigSnapshot() {}
@@ -332,5 +287,26 @@ public StreamElementSerializerConfigSnapshot(TypeSerializer<T> typeSerializer) {
 		public int getVersion() {
 			return VERSION;
 		}
+
+		@Override
+		public int[] getCompatibleVersions() {
+			return new int[]{VERSION, 1};
+		}
+
+		@Override
+		protected boolean containsSerializers() {
+			return getReadVersion() < 2;
+		}
+
+		@SuppressWarnings("unchecked")
+		@Override
+		protected TypeSerializer<StreamElement> restoreSerializer(TypeSerializer<?>[] restoredNestedSerializers) {
+			return new StreamElementSerializer<>((TypeSerializer<T>) restoredNestedSerializers[0]);
+		}
+
+		@Override
+		protected boolean isRecognizableSerializer(TypeSerializer<?> newSerializer) {
+			return newSerializer instanceof StreamElementSerializer;
+		}
 	}
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Allow upgrading to incompatible state serializers (state schema evolution)
> --------------------------------------------------------------------------
>
>                 Key: FLINK-9376
>                 URL: https://issues.apache.org/jira/browse/FLINK-9376
>             Project: Flink
>          Issue Type: New Feature
>          Components: State Backends, Checkpointing, Type Serialization System
>            Reporter: Tzu-Li (Gordon) Tai
>            Priority: Critical
>              Labels: pull-request-available
>
> Currently, users have access to upgrade state serializers on the restore run of a stateful job, as long as the upgraded new serializer remains backwards compatible with all previous written data in the savepoint (i.e. it can read all previous and current schema of serialized state objects).
> What is still lacking is the ability to upgrade to incompatible serializers. Upon being registered an incompatible serializer for existing restored state, that state needs to go through the process of -
>  1. read serialized state with the previous serializer
>  2. passing each deserialized state object through a “migration map function”, and
>  3. writing back the state with the new serializer
> The availability of this process should be strictly limited to state registrations that occur before the actual processing begins (e.g. in the {{open}} or {{initializeState}} methods), so that we avoid performing these operations during processing.
> How this procedure actually occurs, differs across different types of state backends.
> For example, for state backends that eagerly deserialize / lazily serialize state (e.g. {{HeapStateBackend}}), the job execution itself can be seen as a "migration"; everything is deserialized to state objects on restore, and is only serialized again, with the new serializer, on checkpoints.
> Therefore, for these state backends, the above process is irrelevant.
> On the other hand, for state backends that lazily deserialize / eagerly serialize state (e.g. {{RocksDBStateBackend}}), the state evolution process needs to happen for every state with a newly registered incompatible serializer.
> Procedure 2. will allow even state type migrations, but that is out-of-scope of this JIRA.
>  This ticket focuses only on procedures 1. and 3., where we try to enable schema evolution without state type changes.
> This is an umbrella JIRA ticket that overlooks this feature, including a few preliminary tasks that work towards enabling it.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)