From 5fc4a36a144c3f8f22be7e21a4e542d3042d10b1 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Wed, 13 Jun 2018 13:43:53 +0200 Subject: [PATCH 1/5] [FLINK-9377] [core] (part 1) Extend TypeSerializerConfigSnapshot as a factory for restoring serializers This commit is the first step towards removing serializers from checkpointed state meta info and making Flink checkpoints Java serialization free. Instead of writing serializers in checkpoints, and trying to read that to obtain a restore serializer at restore time, we aim to only write the config snapshot as the single source of truth and use it as a factory to create a restore serializer. This commit adds the method and signatures to the TypeSerializerConfigSnapshot interface. Use of the method, as well as properly implementing the method for all serializers, will be implemented in follow-up commits. --- .../typeutils/runtime/WritableSerializer.java | 4 +- ...CompositeTypeSerializerConfigSnapshot.java | 2 +- .../GenericTypeSerializerConfigSnapshot.java | 2 +- .../ParameterlessTypeSerializerConfig.java | 2 +- .../typeutils/TypeDeserializerAdapter.java | 4 +- .../api/common/typeutils/TypeSerializer.java | 6 +- .../TypeSerializerConfigSnapshot.java | 51 +++++++++++-- .../UnloadableDummyTypeSerializer.java | 4 +- .../CollectionSerializerConfigSnapshot.java | 5 +- .../common/typeutils/base/EnumSerializer.java | 2 +- .../base/GenericArraySerializer.java | 4 +- .../GenericArraySerializerConfigSnapshot.java | 2 +- .../common/typeutils/base/ListSerializer.java | 4 +- .../common/typeutils/base/MapSerializer.java | 6 +- .../base/MapSerializerConfigSnapshot.java | 4 +- .../base/TypeSerializerSingleton.java | 8 +- .../runtime/CopyableValueSerializer.java | 4 +- .../typeutils/runtime/EitherSerializer.java | 6 +- .../EitherSerializerConfigSnapshot.java | 2 +- .../typeutils/runtime/PojoSerializer.java | 2 +- .../java/typeutils/runtime/RowSerializer.java | 4 +- .../runtime/TupleSerializerBase.java | 2 +- .../TupleSerializerConfigSnapshot.java | 2 +- .../typeutils/runtime/ValueSerializer.java | 2 +- .../runtime/kryo/KryoSerializer.java | 2 +- .../common/typeutils/SerializerTestBase.java | 4 +- .../TypeSerializerSerializationUtilTest.java | 16 ++-- .../typeutils/runtime/AvroSerializer.java | 2 +- .../avro/typeutils/AvroSerializer.java | 16 ++-- .../BackwardsCompatibleAvroSerializer.java | 4 +- .../java/io/CollectionInputFormatTest.java | 4 +- .../java/org/apache/flink/cep/nfa/NFA.java | 6 +- .../apache/flink/cep/nfa/SharedBuffer.java | 7 +- .../flink/cep/nfa/sharedbuffer/Lockable.java | 29 ++++--- .../LockableSerializerConfigSnapshot.java | 43 +++++++++++ .../ListViewSerializerConfigSnapshot.java | 43 +++++++++++ .../MapViewSerializerConfigSnapshot.java | 44 +++++++++++ .../table/dataview/ListViewSerializer.scala | 55 ++++++++------ .../table/dataview/MapViewSerializer.scala | 68 +++++++++-------- .../table/runtime/types/CRowSerializer.scala | 6 +- .../runtime/state/ArrayListSerializer.java | 6 +- .../testutils/types/IntListSerializer.java | 4 +- .../testutils/types/IntPairSerializer.java | 4 +- .../testutils/types/StringPairSerializer.java | 4 +- .../runtime/query/KvStateRegistryTest.java | 4 +- .../state/OperatorStateBackendTest.java | 4 +- .../runtime/state/StateBackendTestBase.java | 8 +- .../state/heap/CopyOnWriteStateTableTest.java | 4 +- .../recordutils/RecordSerializer.java | 4 +- .../ScalaEitherSerializerConfigSnapshot.java | 46 +++++++++++ .../ScalaOptionSerializerConfigSnapshot.java | 4 +- .../ScalaTrySerializerConfigSnapshot.java | 4 +- .../TraversableSerializerConfigSnapshot.java | 5 +- .../scala/typeutils/EitherSerializer.scala | 76 +++++++++---------- .../scala/typeutils/EnumValueSerializer.scala | 4 +- .../scala/typeutils/NothingSerializer.scala | 4 +- .../scala/typeutils/OptionSerializer.scala | 6 +- .../typeutils/TraversableSerializer.scala | 8 +- .../api/scala/typeutils/TrySerializer.scala | 6 +- .../api/datastream/CoGroupedStreams.java | 9 ++- .../sink/TwoPhaseCommitSinkFunction.java | 8 +- .../api/operators/TimerHeapInternalTimer.java | 4 +- .../streamrecord/StreamElementSerializer.java | 8 +- 63 files changed, 481 insertions(+), 236 deletions(-) create mode 100644 flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/LockableSerializerConfigSnapshot.java create mode 100644 flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/ListViewSerializerConfigSnapshot.java create mode 100644 flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/MapViewSerializerConfigSnapshot.java create mode 100644 flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerConfigSnapshot.java 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 161e65b3b0608..cb6c254f9040a 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 @@ -167,9 +167,9 @@ public WritableSerializerConfigSnapshot snapshotConfiguration() { } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof WritableSerializerConfigSnapshot - && typeClass.equals(((WritableSerializerConfigSnapshot) configSnapshot).getTypeClass())) { + && typeClass.equals(((WritableSerializerConfigSnapshot) configSnapshot).getTypeClass())) { return CompatibilityResult.compatible(); } else { 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 45b78c1a1f06e..287137c6860a8 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,7 +38,7 @@ * is required. */ @Internal -public abstract class CompositeTypeSerializerConfigSnapshot extends TypeSerializerConfigSnapshot { +public abstract class CompositeTypeSerializerConfigSnapshot extends TypeSerializerConfigSnapshot { private List, TypeSerializerConfigSnapshot>> nestedSerializersAndConfigs; 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 4edfe12324134..ae95bfdda0c2f 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 The type to be instantiated. */ @Internal -public abstract class GenericTypeSerializerConfigSnapshot extends TypeSerializerConfigSnapshot { +public abstract class GenericTypeSerializerConfigSnapshot extends TypeSerializerConfigSnapshot { private Class 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 7ba7dd452d609..6fc6d17249172 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 extends TypeSerializerConfigSnapshot { 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 fb59602c5f541..72b25146cf2df 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 snapshotConfiguration() { throw new UnsupportedOperationException( "This is a TypeDeserializerAdapter used only for deserialization; this method should not be used."); } - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult 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 a606a181d1d8f..017e09ebb3811 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,14 +179,14 @@ public abstract class TypeSerializer implements Serializable { * * @return snapshot of the serializer's current configuration (cannot be {@code null}). */ - public abstract TypeSerializerConfigSnapshot snapshotConfiguration(); + public abstract TypeSerializerConfigSnapshot 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 + *

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. @@ -215,5 +215,5 @@ public abstract class TypeSerializer implements Serializable { * * @return the determined compatibility result (cannot be {@code null}). */ - public abstract CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot); + public abstract CompatibilityResult 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 389d141fff0e8..13c8623824127 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: * - *

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: + *

    + *
  • Capturing serializer parameters and schema: a serializer's configuration snapshot + * represents information about the parameters, state, and schema of a serializer. + * This is explained in more detail below.
  • + * + *
  • Compatibility checks for new serializers: 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.
  • + * + *
  • Factory for a read serializer when schema conversion is required: 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.
  • + *
+ * + *

Serializer Configuration and Schema

+ * + *

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

    *
  • Parameter settings of the serializer: parameters of the serializer include settings @@ -38,18 +59,34 @@ * has nested serializers, then the configuration snapshot should also contain the parameters of the nested * serializers.
  • * - *
  • Serialization schema of the serializer: the data format used by the serializer.
  • + *
  • Serialization schema of the serializer: the binary format used by the serializer, or + * in other words, the schema of data written by the serializer.
  • *
* *

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 The data type that the originating serializer of this configuration serializes. */ @PublicEvolving -public abstract class TypeSerializerConfigSnapshot extends VersionedIOReadableWritable { +public abstract class TypeSerializerConfigSnapshot 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 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(); + } + /** * 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/UnloadableDummyTypeSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/UnloadableDummyTypeSerializer.java index ddfeab41c1d37..448c53b209ed4 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 snapshotConfiguration() { throw new UnsupportedOperationException("This object is a dummy TypeSerializer."); } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult 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 55729852ed129..d1a3e9508ac60 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,13 +22,16 @@ 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 Type of the element. */ @Internal -public final class CollectionSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { +public final class CollectionSerializerConfigSnapshot, T> + extends CompositeTypeSerializerConfigSnapshot { private static final int VERSION = 1; 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 c40fefc1694fe..dcd26734f240a 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 @@ -178,7 +178,7 @@ public EnumSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof EnumSerializerConfigSnapshot) { final EnumSerializerConfigSnapshot config = (EnumSerializerConfigSnapshot) configSnapshot; 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 cdfc964e15d6d..a42226cd91146 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 @@ -198,12 +198,12 @@ public String toString() { // -------------------------------------------------------------------------------------------- @Override - public GenericArraySerializerConfigSnapshot snapshotConfiguration() { + public GenericArraySerializerConfigSnapshot snapshotConfiguration() { return new GenericArraySerializerConfigSnapshot<>(componentClass, componentSerializer); } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof GenericArraySerializerConfigSnapshot) { final GenericArraySerializerConfigSnapshot config = (GenericArraySerializerConfigSnapshot) configSnapshot; 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 70e52106b48d6..5b207ddbb2586 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 @@ -36,7 +36,7 @@ * @param The component type. */ @Internal -public final class GenericArraySerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { +public final class GenericArraySerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { private static final int VERSION = 1; 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 c2b935c82e1eb..b5759740c043a 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 @@ -179,7 +179,7 @@ public int hashCode() { // -------------------------------------------------------------------------------------------- @Override - public CollectionSerializerConfigSnapshot snapshotConfiguration() { + public CollectionSerializerConfigSnapshot, T> snapshotConfiguration() { return new CollectionSerializerConfigSnapshot<>(elementSerializer); } @@ -187,7 +187,7 @@ public CollectionSerializerConfigSnapshot snapshotConfiguration() { public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof CollectionSerializerConfigSnapshot) { Tuple2, TypeSerializerConfigSnapshot> previousElemSerializerAndConfig = - ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig(); + ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig(); CompatibilityResult compatResult = CompatibilityUtil.resolveCompatibilityResult( previousElemSerializerAndConfig.f0, 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 6471152b0bc53..570f8f83a2f74 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 @@ -207,15 +207,15 @@ public int hashCode() { // -------------------------------------------------------------------------------------------- @Override - public MapSerializerConfigSnapshot snapshotConfiguration() { + public MapSerializerConfigSnapshot snapshotConfiguration() { return new MapSerializerConfigSnapshot<>(keySerializer, valueSerializer); } @Override - public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof MapSerializerConfigSnapshot) { List, TypeSerializerConfigSnapshot>> previousKvSerializersAndConfigs = - ((MapSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); + ((MapSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); CompatibilityResult keyCompatResult = CompatibilityUtil.resolveCompatibilityResult( previousKvSerializersAndConfigs.get(0).f0, 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 9db3019ad46c5..e601d4d52cca3 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,12 +22,14 @@ 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 extends CompositeTypeSerializerConfigSnapshot { +public final class MapSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot> { private static final int VERSION = 1; 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 9354af07e18a4..13e41675859f7 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 @@ -53,16 +53,16 @@ public boolean equals(Object obj) { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot snapshotConfiguration() { // type serializer singletons should always be parameter-less - return new ParameterlessTypeSerializerConfig(getSerializationFormatIdentifier()); + return new ParameterlessTypeSerializerConfig<>(getSerializationFormatIdentifier()); } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof ParameterlessTypeSerializerConfig && isCompatibleSerializationFormatIdentifier( - ((ParameterlessTypeSerializerConfig) configSnapshot).getSerializationFormatIdentifier())) { + ((ParameterlessTypeSerializerConfig) configSnapshot).getSerializationFormatIdentifier())) { return CompatibilityResult.compatible(); } else { 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 b9039690f9ff3..68e4af3ea446d 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 @@ -142,9 +142,9 @@ public CopyableValueSerializerConfigSnapshot snapshotConfiguration() { } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof CopyableValueSerializerConfigSnapshot - && valueClass.equals(((CopyableValueSerializerConfigSnapshot) configSnapshot).getTypeClass())) { + && valueClass.equals(((CopyableValueSerializerConfigSnapshot) configSnapshot).getTypeClass())) { return CompatibilityResult.compatible(); } else { return CompatibilityResult.requiresMigration(); 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 18ebcd8b63cfd..4f27589e41327 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 @@ -196,15 +196,15 @@ public int hashCode() { // -------------------------------------------------------------------------------------------- @Override - public EitherSerializerConfigSnapshot snapshotConfiguration() { + public EitherSerializerConfigSnapshot snapshotConfiguration() { return new EitherSerializerConfigSnapshot<>(leftSerializer, rightSerializer); } @Override - public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof EitherSerializerConfigSnapshot) { List, TypeSerializerConfigSnapshot>> previousLeftRightSerializersAndConfigs = - ((EitherSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); + ((EitherSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); CompatibilityResult leftCompatResult = CompatibilityUtil.resolveCompatibilityResult( previousLeftRightSerializersAndConfigs.get(0).f0, 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 f9968781a8cb6..628fff4110778 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,7 +28,7 @@ * containing configuration snapshots of the Left and Right serializers. */ @Internal -public final class EitherSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { +public final class EitherSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot> { private static final int VERSION = 1; 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 a4d086d6a7464..cdff1fa0aeca0 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 @@ -594,7 +594,7 @@ public PojoSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof PojoSerializerConfigSnapshot) { final PojoSerializerConfigSnapshot config = (PojoSerializerConfigSnapshot) configSnapshot; 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 7f9cc2145be66..b806c27a97ea9 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 @@ -268,7 +268,7 @@ public RowSerializerConfigSnapshot snapshotConfiguration() { } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof RowSerializerConfigSnapshot) { List, TypeSerializerConfigSnapshot>> previousFieldSerializersAndConfigs = ((RowSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); @@ -312,7 +312,7 @@ public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot return CompatibilityResult.requiresMigration(); } - public static final class RowSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { + public static final class RowSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { private static final int VERSION = 1; 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 3fb7defc2c96c..34146ca8e022a 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 @@ -135,7 +135,7 @@ public TupleSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof TupleSerializerConfigSnapshot) { final TupleSerializerConfigSnapshot config = (TupleSerializerConfigSnapshot) configSnapshot; 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 eac5200da9c6f..a1180aeba5325 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 @@ -32,7 +32,7 @@ * Snapshot of a tuple serializer's configuration. */ @Internal -public final class TupleSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { +public final class TupleSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { private static final int VERSION = 1; 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 0a028ebd79d66..9e02c35901430 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 @@ -182,7 +182,7 @@ public ValueSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof ValueSerializerConfigSnapshot) { final ValueSerializerConfigSnapshot config = (ValueSerializerConfigSnapshot) configSnapshot; 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 d8158aabb92db..5ac914e03ef74 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 @@ -463,7 +463,7 @@ public KryoSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof KryoSerializerConfigSnapshot) { final KryoSerializerConfigSnapshot config = (KryoSerializerConfigSnapshot) configSnapshot; 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 57015c78be0af..d32cb0cd34174 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 @@ -124,7 +124,7 @@ public void testSnapshotConfigurationAndReconfigure() throws Exception { assertFalse(strategy.isRequiresMigration()); // also verify that the serializer's reconfigure implementation detects incompatibility - strategy = getSerializer().ensureCompatibility(new TestIncompatibleSerializerConfigSnapshot()); + strategy = getSerializer().ensureCompatibility(new TestIncompatibleSerializerConfigSnapshot<>()); assertTrue(strategy.isRequiresMigration()); } @@ -526,7 +526,7 @@ public void skipBytesToRead(int numBytes) throws IOException { } } - public static final class TestIncompatibleSerializerConfigSnapshot extends TypeSerializerConfigSnapshot { + public static final class TestIncompatibleSerializerConfigSnapshot extends TypeSerializerConfigSnapshot { @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 04739314dccca..a7136fcd18bd2 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,11 +148,11 @@ public void testSerializerSerializationWithInvalidClass() throws Exception { */ @Test public void testSerializeConfigurationSnapshots() throws Exception { - TypeSerializerSerializationUtilTest.TestConfigSnapshot configSnapshot1 = - new TypeSerializerSerializationUtilTest.TestConfigSnapshot(1, "foo"); + TypeSerializerSerializationUtilTest.TestConfigSnapshot configSnapshot1 = + new TypeSerializerSerializationUtilTest.TestConfigSnapshot<>(1, "foo"); - TypeSerializerSerializationUtilTest.TestConfigSnapshot configSnapshot2 = - new TypeSerializerSerializationUtilTest.TestConfigSnapshot(2, "bar"); + TypeSerializerSerializationUtilTest.TestConfigSnapshot configSnapshot2 = + new TypeSerializerSerializationUtilTest.TestConfigSnapshot<>(2, "bar"); byte[] serializedConfig; try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { @@ -183,7 +183,7 @@ public void testFailsWhenConfigurationSnapshotClassNotFound() throws Exception { byte[] serializedConfig; try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { TypeSerializerSerializationUtil.writeSerializerConfigSnapshot( - new DataOutputViewStreamWrapper(out), new TypeSerializerSerializationUtilTest.TestConfigSnapshot(123, "foobar")); + new DataOutputViewStreamWrapper(out), new TypeSerializerSerializationUtilTest.TestConfigSnapshot<>(123, "foobar")); serializedConfig = out.toByteArray(); } @@ -266,7 +266,7 @@ public void testAnonymousSerializerClassWithChangedSerialVersionUID() throws Exc Assert.assertTrue(anonymousClassSerializer.getClass().isAnonymousClass()); } - public static class TestConfigSnapshot extends TypeSerializerConfigSnapshot { + public static class TestConfigSnapshot extends TypeSerializerConfigSnapshot { static final int VERSION = 1; @@ -411,12 +411,12 @@ public TypeSerializer duplicate() { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot snapshotConfiguration() { return IntSerializer.INSTANCE.snapshotConfiguration(); } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { return IntSerializer.INSTANCE.ensureCompatibility(configSnapshot); } 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 f3801991be6ae..5f76b094361e6 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 @@ -225,7 +225,7 @@ public AvroSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof AvroSerializerConfigSnapshot) { final AvroSerializerConfigSnapshot config = (AvroSerializerConfigSnapshot) configSnapshot; 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 b313625bfe23d..dad1d6df16adf 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 @@ -100,7 +100,7 @@ public class AvroSerializer extends TypeSerializer { private transient Schema schema; /** The serializer configuration snapshot, cached for efficiency. */ - private transient AvroSchemaSerializerConfigSnapshot configSnapshot; + private transient AvroSchemaSerializerConfigSnapshot 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 snapshotConfiguration() { if (configSnapshot == null) { checkAvroInitialized(); - configSnapshot = new AvroSchemaSerializerConfigSnapshot(schema.toString(false)); + configSnapshot = new AvroSchemaSerializerConfigSnapshot<>(schema.toString(false)); } return configSnapshot; } @Override - @SuppressWarnings("deprecation") - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + @SuppressWarnings({"deprecation", "unchecked"}) + public CompatibilityResult 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(); @@ -291,7 +291,7 @@ 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 old = (AvroSerializerConfigSnapshot) configSnapshot; return type.equals(old.getTypeClass()) ? CompatibilityResult.compatible() : CompatibilityResult.requiresMigration(); } @@ -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 extends TypeSerializerConfigSnapshot { 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 e5eb5d89a1f77..addd4b1295423 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 @@ -172,13 +172,13 @@ public String toString() { // ------------------------------------------------------------------------ @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot snapshotConfiguration() { // we return the configuration of the actually used serializer here return serializer.snapshotConfiguration(); } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof AvroSchemaSerializerConfigSnapshot || configSnapshot instanceof AvroSerializerConfigSnapshot) { 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 579e761c014ff..7b1441729b167 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 @@ -395,12 +395,12 @@ public int hashCode() { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot snapshotConfiguration() { throw new UnsupportedOperationException(); } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult 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 041a01702f472..a6ae7746812fa 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 @@ -863,7 +863,7 @@ public Queue getComputationStates() { * The {@link TypeSerializerConfigSnapshot} serializer configuration to be stored with the managed state. */ @Deprecated - public static final class NFASerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { + public static final class NFASerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot> { private static final int VERSION = 1; @@ -987,7 +987,7 @@ public int hashCode() { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot> snapshotConfiguration() { return new NFASerializerConfigSnapshot<>(eventSerializer, sharedBufferSerializer); } @@ -995,7 +995,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof NFASerializerConfigSnapshot) { List, TypeSerializerConfigSnapshot>> serializersAndConfigs = - ((NFASerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); + ((NFASerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); CompatibilityResult eventCompatResult = CompatibilityUtil.resolveCompatibilityResult( serializersAndConfigs.get(0).f0, 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 a4dbc00704da9..4ab227961b0a4 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 @@ -149,7 +149,8 @@ public static ValueTimeWrapper deserialize( /** * The {@link TypeSerializerConfigSnapshot} serializer configuration to be stored with the managed state. */ - public static final class SharedBufferSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { + public static final class SharedBufferSerializerConfigSnapshot + extends CompositeTypeSerializerConfigSnapshot> { private static final int VERSION = 1; @@ -342,7 +343,7 @@ public int hashCode() { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot> snapshotConfiguration() { return new SharedBufferSerializerConfigSnapshot<>( keySerializer, valueSerializer, @@ -353,7 +354,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof SharedBufferSerializerConfigSnapshot) { List, TypeSerializerConfigSnapshot>> serializerConfigSnapshots = - ((SharedBufferSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); + ((SharedBufferSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); CompatibilityResult keyCompatResult = CompatibilityUtil.resolveCompatibilityResult( serializerConfigSnapshots.get(0).f0, 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 b782d8a07c6e7..40b015639731e 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 @@ -19,10 +19,10 @@ 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.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -181,19 +181,28 @@ public boolean canEqual(Object obj) { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { - return elementSerializer.snapshotConfiguration(); + public TypeSerializerConfigSnapshot> snapshotConfiguration() { + return new LockableSerializerConfigSnapshot<>(elementSerializer); } @Override - public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { - CompatibilityResult inputComaptibilityResult = elementSerializer.ensureCompatibility(configSnapshot); - if (inputComaptibilityResult.isRequiresMigration()) { - return CompatibilityResult.requiresMigration(new LockableTypeSerializer<>( - new TypeDeserializerAdapter<>(inputComaptibilityResult.getConvertDeserializer())) - ); + public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof LockableSerializerConfigSnapshot) { + @SuppressWarnings("unchecked") + LockableSerializerConfigSnapshot snapshot = (LockableSerializerConfigSnapshot) configSnapshot; + + Tuple2, TypeSerializerConfigSnapshot> nestedSerializerAndConfig = + snapshot.getSingleNestedSerializerAndConfig(); + + CompatibilityResult inputComaptibilityResult = elementSerializer + .internalEnsureCompatibility(nestedSerializerAndConfig.f1); + if (inputComaptibilityResult.isRequiresMigration()) { + return CompatibilityResult.requiresMigration(); + } else { + return CompatibilityResult.compatible(); + } } else { - return CompatibilityResult.compatible(); + return CompatibilityResult.requiresMigration(); } } } 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 0000000000000..f3702ac0459a7 --- /dev/null +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/sharedbuffer/LockableSerializerConfigSnapshot.java @@ -0,0 +1,43 @@ +/* + * 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 extends CompositeTypeSerializerConfigSnapshot> { + + private static final int VERSION = 1; + + public LockableSerializerConfigSnapshot(TypeSerializer elementSerializer) { + super(elementSerializer); + } + + @Override + public int getVersion() { + return VERSION; + } + +} 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 0000000000000..6230e811c4658 --- /dev/null +++ b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/ListViewSerializerConfigSnapshot.java @@ -0,0 +1,43 @@ +/* + * 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.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 the type of the list elements. + */ +public final class ListViewSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot> { + + private static final int VERSION = 1; + + public ListViewSerializerConfigSnapshot(ListSerializer listSerializer) { + super(listSerializer); + } + + @Override + public int getVersion() { + return VERSION; + } +} 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 0000000000000..36954b95b236a --- /dev/null +++ b/flink-libraries/flink-table/src/main/java/org/apache/flink/table/dataview/MapViewSerializerConfigSnapshot.java @@ -0,0 +1,44 @@ +/* + * 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.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 the key type of the map entries. + * @param the value type of the map entries. + */ +public class MapViewSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot> { + + private static final int VERSION = 1; + + public MapViewSerializerConfigSnapshot(MapSerializer mapSerializer) { + super(mapSerializer); + } + + @Override + public int getVersion() { + return VERSION; + } +} 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 a450c2ce1e552..71ef183943f27 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 @@ -75,37 +75,42 @@ 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() + override def snapshotConfiguration(): ListViewSerializerConfigSnapshot[T] = + new ListViewSerializerConfigSnapshot[T](listSerializer) - // copy and modified from ListSerializer.ensureCompatibility override def ensureCompatibility( - configSnapshot: TypeSerializerConfigSnapshot): CompatibilityResult[ListView[T]] = { + 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 snapshot: ListViewSerializerConfigSnapshot[T] => + checkCompatibility(snapshot) + + // backwards compatibility path; + // Flink versions older or equal to 1.5.x returns a + // CollectionSerializerConfigSnapshot as the snapshot + case legacySnapshot: CollectionSerializerConfigSnapshot[java.util.List[T], T] => + checkCompatibility(legacySnapshot) case _ => CompatibilityResult.requiresMigration[ListView[T]] } } + + private def checkCompatibility( + configSnapshot: CompositeTypeSerializerConfigSnapshot[_] + ): CompatibilityResult[ListView[T]] = { + + val previousListSerializerAndConfig = configSnapshot.getSingleNestedSerializerAndConfig + + val compatResult = CompatibilityUtil.resolveCompatibilityResult( + previousListSerializerAndConfig.f0, + classOf[UnloadableDummyTypeSerializer[_]], + previousListSerializerAndConfig.f1, + listSerializer.getElementSerializer) + + if (!compatResult.isRequiresMigration) { + CompatibilityResult.compatible[ListView[T]] + } else { + CompatibilityResult.requiresMigration[ListView[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 c53f10c37e597..0a98e34ede0d2 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 @@ -77,45 +77,49 @@ 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() + override def snapshotConfiguration(): MapViewSerializerConfigSnapshot[K, V] = + new MapViewSerializerConfigSnapshot[K, V](mapSerializer) // copy and modified from MapSerializer.ensureCompatibility - override def ensureCompatibility(configSnapshot: TypeSerializerConfigSnapshot) + 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 snapshot: MapViewSerializerConfigSnapshot[K, V] => + checkCompatibility(snapshot) + + // backwards compatibility path; + // Flink versions older or equal to 1.5.x returns a + // MapSerializerConfigSnapshot as the snapshot + case legacySnapshot: MapSerializerConfigSnapshot[K, V] => + checkCompatibility(legacySnapshot) case _ => CompatibilityResult.requiresMigration[MapView[K, V]] } } + + private def checkCompatibility( + configSnapshot: CompositeTypeSerializerConfigSnapshot[_] + ): CompatibilityResult[MapView[K, V]] = { + + val previousKvSerializersAndConfigs = configSnapshot.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 { + CompatibilityResult.requiresMigration[MapView[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 caf346c3430bd..e96da82dc3bb9 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,12 +80,12 @@ 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[_]): CompatibilityResult[CRow] = { configSnapshot match { case crowSerializerConfigSnapshot: CRowSerializer.CRowSerializerConfigSnapshot => @@ -117,7 +117,7 @@ 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) 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 3c4f4b0a26949..1868abf311c16 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 @@ -145,15 +145,15 @@ public int hashCode() { // -------------------------------------------------------------------------------------------- @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot> snapshotConfiguration() { return new CollectionSerializerConfigSnapshot<>(elementSerializer); } @Override - public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof CollectionSerializerConfigSnapshot) { Tuple2, TypeSerializerConfigSnapshot> previousElemSerializerAndConfig = - ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig(); + ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig(); CompatibilityResult compatResult = CompatibilityUtil.resolveCompatibilityResult( previousElemSerializerAndConfig.f0, 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 4bdc5e85cdd31..67d8a26c81357 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 @@ -131,12 +131,12 @@ public int hashCode() { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot snapshotConfiguration() { throw new UnsupportedOperationException(); } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult 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 0ae5e71d42803..fe456c8204cfc 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 @@ -140,12 +140,12 @@ public boolean equals(Object obj) { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot snapshotConfiguration() { throw new UnsupportedOperationException(); } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult 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 17ee5f1530b4a..cad78e7c01817 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 @@ -108,12 +108,12 @@ public int hashCode() { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot snapshotConfiguration() { throw new UnsupportedOperationException(); } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult 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 c1c56bf250b87..82d89e05a4dcd 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 @@ -401,12 +401,12 @@ public int hashCode() { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot snapshotConfiguration() { return null; } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { return null; } } 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 d8918e784787b..bbf7fd5676382 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 @@ -351,12 +351,12 @@ public int hashCode() { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot snapshotConfiguration() { return IntSerializer.INSTANCE.snapshotConfiguration(); } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { return IntSerializer.INSTANCE.ensureCompatibility(configSnapshot); } } 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 ad67171a01c87..34ba80ee2319a 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 @@ -4242,14 +4242,14 @@ private TestReconfigurableCustomTypeSerializer(boolean reconfigured) { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { - return new ParameterlessTypeSerializerConfig(getClass().getName()); + public TypeSerializerConfigSnapshot snapshotConfiguration() { + return new ParameterlessTypeSerializerConfig<>(getClass().getName()); } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult 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(); 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 4f36d62592249..dbccec1a3fddf 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 @@ -654,12 +654,12 @@ public void disable() { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot snapshotConfiguration() { throw new UnsupportedOperationException(); } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult 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 ce23f30006be4..87d91e2ea0ea8 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 @@ -145,12 +145,12 @@ public int hashCode() { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot snapshotConfiguration() { throw new UnsupportedOperationException(); } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult 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 0000000000000..9566a663169b0 --- /dev/null +++ b/flink-scala/src/main/java/org/apache/flink/api/scala/typeutils/ScalaEitherSerializerConfigSnapshot.java @@ -0,0 +1,46 @@ +/* + * 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, L, R> + extends CompositeTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public ScalaEitherSerializerConfigSnapshot() {} + + public ScalaEitherSerializerConfigSnapshot(TypeSerializer leftSerializer, TypeSerializer rightSerializer) { + super(leftSerializer, rightSerializer); + } + + @Override + public int getVersion() { + return VERSION; + } +} 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 03eef12e9ca4a..215bd447c05d4 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,7 +31,7 @@ * allow calling different base class constructors from subclasses, while we need that * for the default empty constructor. */ -public final class ScalaOptionSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { +public final class ScalaOptionSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot> { private static final int VERSION = 1; 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 6abb3ea06d761..72baca44193bb 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,7 +31,7 @@ * allow calling different base class constructors from subclasses, while we need that * for the default empty constructor. */ -public class ScalaTrySerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { +public class ScalaTrySerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot> { private static final int VERSION = 1; 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 9a39421dad58a..b7f9ca6ec1afb 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 extends CompositeTypeSerializerConfigSnapshot { +public class TraversableSerializerConfigSnapshot, E> + extends CompositeTypeSerializerConfigSnapshot { private static final int VERSION = 1; 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 439e0c2865d16..82637befdcd1a 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 @@ -111,51 +111,51 @@ 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 snapshotConfiguration(): ScalaEitherSerializerConfigSnapshot[T, A, B] = { + new ScalaEitherSerializerConfigSnapshot[T, A, B](leftSerializer, rightSerializer) } override def ensureCompatibility( - configSnapshot: TypeSerializerConfigSnapshot): CompatibilityResult[T] = { + 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 eitherSerializerConfig: ScalaEitherSerializerConfigSnapshot[T, A, B] => + checkCompatibility(eitherSerializerConfig) + + // backwards compatibility path; + // Flink versions older or equal to 1.5.x uses a + // EitherSerializerConfigSnapshot as the snapshot + case legacyConfig: EitherSerializerConfigSnapshot[A, B] => + checkCompatibility(legacyConfig) case _ => CompatibilityResult.requiresMigration() } } + + private def checkCompatibility( + configSnapshot: CompositeTypeSerializerConfigSnapshot[_] + ): CompatibilityResult[T] = { + + val previousLeftRightSerWithConfigs = + configSnapshot.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) { + CompatibilityResult.requiresMigration() + } else { + CompatibilityResult.compatible() + } + } } 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 abc56d133e074..b7e21ceaaee2a 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 @@ -84,7 +84,7 @@ class EnumValueSerializer[E <: Enumeration](val enum: E) extends TypeSerializer[ } override def ensureCompatibility( - configSnapshot: TypeSerializerConfigSnapshot): CompatibilityResult[E#Value] = { + configSnapshot: TypeSerializerConfigSnapshot[_]): CompatibilityResult[E#Value] = { configSnapshot match { case enumSerializerConfigSnapshot: EnumValueSerializer.ScalaEnumSerializerConfigSnapshot[_] => @@ -122,7 +122,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 01ca29594e2a2..eff57b68755b7 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 @@ -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[_]): CompatibilityResult[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 aa4a0ea75b8f9..a34826f2765ee 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 @@ -106,7 +106,7 @@ class OptionSerializer[A](val elemSerializer: TypeSerializer[A]) } override def ensureCompatibility( - configSnapshot: TypeSerializerConfigSnapshot): CompatibilityResult[Option[A]] = { + configSnapshot: TypeSerializerConfigSnapshot[_]): CompatibilityResult[Option[A]] = { configSnapshot match { case optionSerializerConfigSnapshot @@ -120,7 +120,7 @@ class OptionSerializer[A](val elemSerializer: TypeSerializer[A]) } private def ensureCompatibility( - compositeConfigSnapshot: CompositeTypeSerializerConfigSnapshot) + compositeConfigSnapshot: CompositeTypeSerializerConfigSnapshot[Option[A]]) : CompatibilityResult[Option[A]] = { val compatResult = CompatibilityUtil.resolveCompatibilityResult( @@ -150,7 +150,7 @@ 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 } 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 b54193b48291f..b5d069fbe9396 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,16 +152,16 @@ 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[_]): CompatibilityResult[T] = { configSnapshot match { case traversableSerializerConfigSnapshot - : TraversableSerializerConfigSnapshot[E] => + : TraversableSerializerConfigSnapshot[T, E] => val elemCompatRes = CompatibilityUtil.resolveCompatibilityResult( traversableSerializerConfigSnapshot.getSingleNestedSerializerAndConfig.f0, 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 cc9c5ccede775..0e5fd97acf16e 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 @@ -109,7 +109,7 @@ class TrySerializer[A]( } override def ensureCompatibility( - configSnapshot: TypeSerializerConfigSnapshot): CompatibilityResult[Try[A]] = { + configSnapshot: TypeSerializerConfigSnapshot[_]): CompatibilityResult[Try[A]] = { configSnapshot match { case trySerializerConfigSnapshot @@ -123,7 +123,7 @@ class TrySerializer[A]( } private def ensureCompatibility( - compositeConfigSnapshot: CompositeTypeSerializerConfigSnapshot) + compositeConfigSnapshot: CompositeTypeSerializerConfigSnapshot[Try[A]]) : CompatibilityResult[Try[A]] = { val previousSerializersAndConfigs = @@ -156,7 +156,7 @@ 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 } 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 c2ebdf483ed11..5b19cbe3915a8 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 @@ -553,15 +553,15 @@ public boolean canEqual(Object obj) { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot> snapshotConfiguration() { return new UnionSerializerConfigSnapshot<>(oneSerializer, twoSerializer); } @Override - public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof UnionSerializerConfigSnapshot) { List, TypeSerializerConfigSnapshot>> previousSerializersAndConfigs = - ((UnionSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); + ((UnionSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); CompatibilityResult oneSerializerCompatResult = CompatibilityUtil.resolveCompatibilityResult( previousSerializersAndConfigs.get(0).f0, @@ -592,7 +592,8 @@ public CompatibilityResult> ensureCompatibility(TypeSerializ /** * The {@link TypeSerializerConfigSnapshot} for the {@link UnionSerializer}. */ - public static class UnionSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { + public static class UnionSerializerConfigSnapshot + extends CompositeTypeSerializerConfigSnapshot> { private static final int VERSION = 1; 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 2ffb6d5810ed5..275cfc7634447 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 @@ -763,16 +763,16 @@ public int hashCode() { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot> snapshotConfiguration() { return new StateSerializerConfigSnapshot<>(transactionSerializer, contextSerializer); } @Override public CompatibilityResult> ensureCompatibility( - TypeSerializerConfigSnapshot configSnapshot) { + TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof StateSerializerConfigSnapshot) { List, TypeSerializerConfigSnapshot>> previousSerializersAndConfigs = - ((StateSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); + ((StateSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); CompatibilityResult txnCompatResult = CompatibilityUtil.resolveCompatibilityResult( previousSerializersAndConfigs.get(0).f0, @@ -809,7 +809,7 @@ public CompatibilityResult> ensureCompatibility( */ @Internal public static final class StateSerializerConfigSnapshot - extends CompositeTypeSerializerConfigSnapshot { + extends CompositeTypeSerializerConfigSnapshot> { private static final int VERSION = 1; 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 906b0908abbf7..dd2709bfc4dc2 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 @@ -242,12 +242,12 @@ public int hashCode() { } @Override - public TypeSerializerConfigSnapshot snapshotConfiguration() { + public TypeSerializerConfigSnapshot> snapshotConfiguration() { throw new UnsupportedOperationException("This serializer is not registered for managed state."); } @Override - public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult> 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 ba92416d792ce..635851ed259c2 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 @@ -281,18 +281,18 @@ public int hashCode() { // -------------------------------------------------------------------------------------------- @Override - public StreamElementSerializerConfigSnapshot snapshotConfiguration() { + public StreamElementSerializerConfigSnapshot snapshotConfiguration() { return new StreamElementSerializerConfigSnapshot<>(typeSerializer); } @Override - public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { Tuple2, TypeSerializerConfigSnapshot> previousTypeSerializerAndConfig; // we are compatible for data written by ourselves or the legacy MultiplexingStreamRecordSerializer if (configSnapshot instanceof StreamElementSerializerConfigSnapshot) { previousTypeSerializerAndConfig = - ((StreamElementSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig(); + ((StreamElementSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig(); } else { return CompatibilityResult.requiresMigration(); } @@ -317,7 +317,7 @@ public CompatibilityResult ensureCompatibility(TypeSerializerConf /** * Configuration snapshot specific to the {@link StreamElementSerializer}. */ - public static final class StreamElementSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { + public static final class StreamElementSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { private static final int VERSION = 1; From 661eb6d34da450ed096a77f166a4cc62ce3efdba Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Thu, 14 Jun 2018 11:52:06 +0200 Subject: [PATCH 2/5] [FLINK-9377] [core] (part 2) Remove fallback deserializer option from CompatibilityResult Now that the config snapshot is used as a factory for the restore serializer, it should be guaranteed that a restore serializer is always available. This removes the need for the user to provide a "fallback" convert serializer in the case where a migration is required. --- .../common/typeutils/CompatibilityResult.java | 68 ++----------------- .../common/typeutils/CompatibilityUtil.java | 21 +----- .../base/GenericArraySerializer.java | 9 --- .../common/typeutils/base/ListSerializer.java | 7 -- .../common/typeutils/base/MapSerializer.java | 11 --- .../typeutils/runtime/EitherSerializer.java | 13 ---- .../typeutils/runtime/PojoSerializer.java | 54 +++------------ .../java/typeutils/runtime/RowSerializer.java | 26 +------ .../runtime/TupleSerializerBase.java | 26 +------ .../java/org/apache/flink/cep/nfa/NFA.java | 24 ++----- .../apache/flink/cep/nfa/SharedBuffer.java | 31 ++------- .../table/dataview/ListViewSerializer.scala | 2 - .../table/dataview/MapViewSerializer.scala | 4 -- .../table/runtime/types/CRowSerializer.scala | 11 +-- .../runtime/state/ArrayListSerializer.java | 7 -- .../state/DefaultOperatorStateBackend.java | 6 -- .../RegisteredKeyedBackendStateMetaInfo.java | 5 -- .../state/heap/HeapKeyedStateBackend.java | 3 - .../scala/typeutils/EitherSerializer.scala | 4 -- .../scala/typeutils/OptionSerializer.scala | 10 +-- .../typeutils/TraversableSerializer.scala | 2 - .../api/scala/typeutils/TrySerializer.scala | 4 -- .../state/RocksDBKeyedStateBackend.java | 5 -- .../api/datastream/CoGroupedStreams.java | 11 --- .../sink/TwoPhaseCommitSinkFunction.java | 13 ---- .../operators/HeapInternalTimerService.java | 4 -- .../streamrecord/StreamElementSerializer.java | 8 --- 27 files changed, 36 insertions(+), 353 deletions(-) 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 index 1e05d57854b96..992b54a7dbf76 100644 --- 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 @@ -19,9 +19,6 @@ 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 @@ -35,79 +32,26 @@ public final class CompatibilityResult { /** 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. - * - *

This is only relevant if migration is required. - */ - private final TypeDeserializer 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 + * @return a result that signals migration is not required for the new serializer. */ public static CompatibilityResult 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 the type of the data being migrated. - * - * @return a result that signals migration is necessary, also providing a convert deserializer. - */ - public static CompatibilityResult requiresMigration(@Nonnull TypeDeserializer 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 the type of the data being migrated. - * - * @return a result that signals migration is necessary, also providing a convert serializer. - */ - public static CompatibilityResult requiresMigration(@Nonnull TypeSerializer convertSerializer) { - Preconditions.checkNotNull(convertSerializer, "Convert serializer cannot be null."); - - return new CompatibilityResult<>(true, new TypeDeserializerAdapter<>(convertSerializer)); + return new CompatibilityResult<>(false); } /** - * Returns a result that signals migration to be performed. The migration will fail if the preceding - * serializer for the previous data cannot be found. + * Returns a result that signals migration to be performed. * - *

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. + * @return a result that signals migration is necessary. */ public static CompatibilityResult requiresMigration() { - return new CompatibilityResult<>(true, null); + return new CompatibilityResult<>(true); } - private CompatibilityResult(boolean requiresMigration, TypeDeserializer convertDeserializer) { + private CompatibilityResult(boolean requiresMigration) { this.requiresMigration = requiresMigration; - this.convertDeserializer = convertDeserializer; - } - - public TypeDeserializer getConvertDeserializer() { - return convertDeserializer; } public boolean isRequiresMigration() { 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 6c8583c196815..cff464943b934 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 @@ public class CompatibilityUtil { * 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 * @@ -55,26 +51,11 @@ public class CompatibilityUtil { */ @SuppressWarnings("unchecked") public static CompatibilityResult resolveCompatibilityResult( - @Nullable TypeSerializer precedingSerializer, - Class dummySerializerClassTag, TypeSerializerConfigSnapshot precedingSerializerConfigSnapshot, TypeSerializer newSerializer) { if (precedingSerializerConfigSnapshot != null) { - CompatibilityResult 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) precedingSerializer); - } else { - // requires migration (may or may not have a convert deserializer) - return initialResult; - } - } + return newSerializer.ensureCompatibility(precedingSerializerConfigSnapshot); } else { // if the configuration snapshot of the preceding serializer cannot be provided, // we can only simply assume that the new serializer is compatible 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 a42226cd91146..8e0969f350929 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 @@ -24,10 +24,8 @@ 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.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; @@ -212,18 +210,11 @@ public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot config.getSingleNestedSerializerAndConfig(); CompatibilityResult 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()))); } } } 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 b5759740c043a..1ae0250ce2802 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 @@ -21,10 +21,8 @@ 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.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; @@ -190,16 +188,11 @@ public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSnap ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig(); CompatibilityResult 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()))); } } 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 570f8f83a2f74..e010a905281ae 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 @@ -21,10 +21,8 @@ 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.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; @@ -218,24 +216,15 @@ public CompatibilityResult> ensureCompatibility(TypeSerializerConfigSn ((MapSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); CompatibilityResult keyCompatResult = CompatibilityUtil.resolveCompatibilityResult( - previousKvSerializersAndConfigs.get(0).f0, - UnloadableDummyTypeSerializer.class, previousKvSerializersAndConfigs.get(0).f1, keySerializer); CompatibilityResult 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()))); } } 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 4f27589e41327..4456b4de83286 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 @@ -21,10 +21,8 @@ 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.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; @@ -207,26 +205,15 @@ public CompatibilityResult> ensureCompatibility(TypeSerializerConfi ((EitherSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); CompatibilityResult leftCompatResult = CompatibilityUtil.resolveCompatibilityResult( - previousLeftRightSerializersAndConfigs.get(0).f0, - UnloadableDummyTypeSerializer.class, previousLeftRightSerializersAndConfigs.get(0).f1, leftSerializer); CompatibilityResult 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()))); - } } } 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 cdff1fa0aeca0..39b91245b419d 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 @@ -39,7 +39,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; 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; @@ -598,8 +597,6 @@ public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot config = (PojoSerializerConfigSnapshot) configSnapshot; - boolean requiresMigration = false; - if (clazz.equals(config.getTypeClass())) { if (this.numFields == config.getFieldToSerializerConfigSnapshot().size()) { @@ -622,19 +619,11 @@ public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot) compatResult.getConvertDeserializer(); - } else { - return CompatibilityResult.requiresMigration(); - } + return CompatibilityResult.requiresMigration(); } else { reorderedFieldSerializers[i] = fieldSerializers[fieldIndex]; } @@ -671,17 +660,11 @@ public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot, 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(); - } + return CompatibilityResult.requiresMigration(); } i++; @@ -700,19 +683,11 @@ public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot ensureCompatibility(TypeSerializerConfigSnapshot( - clazz, - reorderedFields, - reorderedFieldSerializers, - reorderedRegisteredSubclassesToClasstags, - reorderedRegisteredSubclassSerializers, - rebuiltCache)); - } + return CompatibilityResult.compatible(); } } } 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 b806c27a97ea9..a7c3b18a4ae4f 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 @@ -21,10 +21,8 @@ import org.apache.flink.api.common.typeutils.CompatibilityResult; import org.apache.flink.api.common.typeutils.CompatibilityUtil; 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; @@ -274,38 +272,20 @@ public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot ((RowSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); if (previousFieldSerializersAndConfigs.size() == fieldSerializers.length) { - boolean requireMigration = false; - TypeSerializer[] convertDeserializers = new TypeSerializer[fieldSerializers.length]; CompatibilityResult compatResult; int i = 0; for (Tuple2, TypeSerializerConfigSnapshot> f : previousFieldSerializersAndConfigs) { - compatResult = CompatibilityUtil.resolveCompatibilityResult( - f.f0, - UnloadableDummyTypeSerializer.class, - f.f1, - fieldSerializers[i]); + compatResult = CompatibilityUtil.resolveCompatibilityResult(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()); - } + return CompatibilityResult.requiresMigration(); } i++; } - if (requireMigration) { - return CompatibilityResult.requiresMigration(new RowSerializer(convertDeserializers)); - } else { - return CompatibilityResult.compatible(); - } + return CompatibilityResult.compatible(); } } 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 34146ca8e022a..fdcd828ca3643 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 @@ -22,10 +22,8 @@ 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.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; @@ -145,37 +143,19 @@ public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot[] convertFieldSerializers = new TypeSerializer[fieldSerializers.length]; - boolean requiresMigration = false; CompatibilityResult compatResult; int i = 0; for (Tuple2, TypeSerializerConfigSnapshot> f : previousFieldSerializersAndConfigs) { - compatResult = CompatibilityUtil.resolveCompatibilityResult( - f.f0, - UnloadableDummyTypeSerializer.class, - f.f1, - fieldSerializers[i]); + compatResult = CompatibilityUtil.resolveCompatibilityResult(f.f1, fieldSerializers[i]); if (compatResult.isRequiresMigration()) { - requiresMigration = true; - - if (compatResult.getConvertDeserializer() != null) { - convertFieldSerializers[i] = - new TypeDeserializerAdapter<>(compatResult.getConvertDeserializer()); - } else { - return CompatibilityResult.requiresMigration(); - } + return CompatibilityResult.requiresMigration(); } i++; } - if (!requiresMigration) { - return CompatibilityResult.compatible(); - } else { - return CompatibilityResult.requiresMigration( - createSerializerInstance(tupleClass, convertFieldSerializers)); - } + return CompatibilityResult.compatible(); } } } 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 a6ae7746812fa..1fe4a865fae7c 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 @@ -22,10 +22,8 @@ import org.apache.flink.api.common.typeutils.CompatibilityResult; import org.apache.flink.api.common.typeutils.CompatibilityUtil; 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; @@ -998,28 +996,16 @@ public CompatibilityResult> ensureCompatibility(TypeSerializerCon ((NFASerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); CompatibilityResult eventCompatResult = CompatibilityUtil.resolveCompatibilityResult( - serializersAndConfigs.get(0).f0, - UnloadableDummyTypeSerializer.class, - serializersAndConfigs.get(0).f1, - eventSerializer); + serializersAndConfigs.get(0).f1, + eventSerializer); CompatibilityResult> sharedBufCompatResult = - CompatibilityUtil.resolveCompatibilityResult( - serializersAndConfigs.get(1).f0, - UnloadableDummyTypeSerializer.class, - serializersAndConfigs.get(1).f1, - sharedBufferSerializer); + CompatibilityUtil.resolveCompatibilityResult( + 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()))); - } } } 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 4ab227961b0a4..6fc50bc86bd32 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 @@ -21,10 +21,8 @@ import org.apache.flink.api.common.typeutils.CompatibilityResult; import org.apache.flink.api.common.typeutils.CompatibilityUtil; 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; @@ -357,37 +355,22 @@ public CompatibilityResult> ensureCompatibility(TypeSerializerCo ((SharedBufferSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); CompatibilityResult keyCompatResult = CompatibilityUtil.resolveCompatibilityResult( - serializerConfigSnapshots.get(0).f0, - UnloadableDummyTypeSerializer.class, - serializerConfigSnapshots.get(0).f1, - keySerializer); + serializerConfigSnapshots.get(0).f1, + keySerializer); CompatibilityResult valueCompatResult = CompatibilityUtil.resolveCompatibilityResult( - serializerConfigSnapshots.get(1).f0, - UnloadableDummyTypeSerializer.class, - serializerConfigSnapshots.get(1).f1, - valueSerializer); + serializerConfigSnapshots.get(1).f1, + valueSerializer); CompatibilityResult versionCompatResult = CompatibilityUtil.resolveCompatibilityResult( - serializerConfigSnapshots.get(2).f0, - UnloadableDummyTypeSerializer.class, - serializerConfigSnapshots.get(2).f1, - versionSerializer); + 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-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 71ef183943f27..b8f631fd067f1 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 @@ -102,8 +102,6 @@ class ListViewSerializer[T](val listSerializer: ListSerializer[T]) val previousListSerializerAndConfig = configSnapshot.getSingleNestedSerializerAndConfig val compatResult = CompatibilityUtil.resolveCompatibilityResult( - previousListSerializerAndConfig.f0, - classOf[UnloadableDummyTypeSerializer[_]], previousListSerializerAndConfig.f1, listSerializer.getElementSerializer) 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 0a98e34ede0d2..d624a40944c85 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 @@ -105,14 +105,10 @@ class MapViewSerializer[K, V](val mapSerializer: MapSerializer[K, V]) val previousKvSerializersAndConfigs = configSnapshot.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) 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 e96da82dc3bb9..a7f1f4748bb8d 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 @@ -90,20 +90,11 @@ class CRowSerializer(val rowSerializer: TypeSerializer[Row]) extends TypeSeriali configSnapshot match { case crowSerializerConfigSnapshot: CRowSerializer.CRowSerializerConfigSnapshot => val compatResult = CompatibilityUtil.resolveCompatibilityResult( - crowSerializerConfigSnapshot.getSingleNestedSerializerAndConfig.f0, - classOf[UnloadableDummyTypeSerializer[_]], crowSerializerConfigSnapshot.getSingleNestedSerializerAndConfig.f1, rowSerializer) if (compatResult.isRequiresMigration) { - if (compatResult.getConvertDeserializer != null) { - CompatibilityResult.requiresMigration( - new CRowSerializer( - new TypeDeserializerAdapter(compatResult.getConvertDeserializer)) - ) - } else { - CompatibilityResult.requiresMigration() - } + CompatibilityResult.requiresMigration() } else { CompatibilityResult.compatible() } 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 1868abf311c16..d30864c6c5d89 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 @@ -19,10 +19,8 @@ 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.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; @@ -156,16 +154,11 @@ public CompatibilityResult> ensureCompatibility(TypeSerializerConfi ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerAndConfig(); CompatibilityResult 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()))); } } 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 a2e49cb179d7f..58083b8c87612 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 @@ -226,14 +226,10 @@ public BroadcastState getBroadcastState(final MapStateDescriptor keyCompatibility = CompatibilityUtil.resolveCompatibilityResult( - restoredMetaInfo.getKeySerializer(), - UnloadableDummyTypeSerializer.class, restoredMetaInfo.getKeySerializerConfigSnapshot(), broadcastStateKeySerializer); CompatibilityResult valueCompatibility = CompatibilityUtil.resolveCompatibilityResult( - restoredMetaInfo.getValueSerializer(), - UnloadableDummyTypeSerializer.class, restoredMetaInfo.getValueSerializerConfigSnapshot(), broadcastStateValueSerializer); @@ -759,8 +755,6 @@ private ListState getListState( // check compatibility to determine if state migration is required TypeSerializer newPartitionStateSerializer = partitionStateSerializer.duplicate(); CompatibilityResult stateCompatibility = CompatibilityUtil.resolveCompatibilityResult( - restoredMetaInfo.getPartitionStateSerializer(), - UnloadableDummyTypeSerializer.class, restoredMetaInfo.getPartitionStateSerializerConfigSnapshot(), newPartitionStateSerializer); 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 c7952ef985ca6..3f7a8d6f8247e 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 @@ -23,7 +23,6 @@ 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; @@ -273,15 +272,11 @@ public static RegisteredKeyedBackendStateMetaInfo resolveKvStateCo // check compatibility results to determine if state migration is required CompatibilityResult namespaceCompatibility = CompatibilityUtil.resolveCompatibilityResult( - restoredStateMetaInfoSnapshot.getNamespaceSerializer(), - null, restoredStateMetaInfoSnapshot.getNamespaceSerializerConfigSnapshot(), newNamespaceSerializer); TypeSerializer newStateSerializer = newStateDescriptor.getSerializer(); CompatibilityResult stateCompatibility = CompatibilityUtil.resolveCompatibilityResult( - restoredStateMetaInfoSnapshot.getStateSerializer(), - UnloadableDummyTypeSerializer.class, restoredStateMetaInfoSnapshot.getStateSerializerConfigSnapshot(), newStateSerializer); 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 82ce5847627a1..87c3b57df541c 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; @@ -292,8 +291,6 @@ private void restorePartitionedState(Collection 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()) { 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 82637befdcd1a..5049da45eb92e 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 @@ -140,14 +140,10 @@ class EitherSerializer[A, B, T <: Either[A, B]]( configSnapshot.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) 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 a34826f2765ee..c7ac4e95081d6 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 @@ -124,19 +124,11 @@ class OptionSerializer[A](val elemSerializer: TypeSerializer[A]) : 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() - } + CompatibilityResult.requiresMigration() } else { CompatibilityResult.compatible() } 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 b5d069fbe9396..574dc0474c4aa 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 @@ -164,8 +164,6 @@ abstract class TraversableSerializer[T <: TraversableOnce[E], E]( : TraversableSerializerConfigSnapshot[T, E] => val elemCompatRes = CompatibilityUtil.resolveCompatibilityResult( - traversableSerializerConfigSnapshot.getSingleNestedSerializerAndConfig.f0, - classOf[UnloadableDummyTypeSerializer[_]], traversableSerializerConfigSnapshot.getSingleNestedSerializerAndConfig.f1, elementSerializer) 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 0e5fd97acf16e..8f4c2a2c3447d 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 @@ -130,14 +130,10 @@ class TrySerializer[A]( 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) 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 e5f443a95ce8b..a7b8b4cb4580d 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,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.common.typeutils.base.array.BytePrimitiveArraySerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ConfigConstants; @@ -612,8 +611,6 @@ 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()) { @@ -1156,8 +1153,6 @@ 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()) { 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 5b19cbe3915a8..01122c9e215f0 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 @@ -27,10 +27,8 @@ import org.apache.flink.api.common.typeutils.CompatibilityResult; import org.apache.flink.api.common.typeutils.CompatibilityUtil; 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; @@ -564,24 +562,15 @@ public CompatibilityResult> ensureCompatibility(TypeSerializ ((UnionSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); CompatibilityResult oneSerializerCompatResult = CompatibilityUtil.resolveCompatibilityResult( - previousSerializersAndConfigs.get(0).f0, - UnloadableDummyTypeSerializer.class, previousSerializersAndConfigs.get(0).f1, oneSerializer); CompatibilityResult 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()))); } } 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 275cfc7634447..3e41ae23aa621 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 @@ -27,10 +27,8 @@ import org.apache.flink.api.common.typeutils.CompatibilityResult; import org.apache.flink.api.common.typeutils.CompatibilityUtil; 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; @@ -775,26 +773,15 @@ public CompatibilityResult> ensureCompatibility( ((StateSerializerConfigSnapshot) configSnapshot).getNestedSerializersAndConfigs(); CompatibilityResult txnCompatResult = CompatibilityUtil.resolveCompatibilityResult( - previousSerializersAndConfigs.get(0).f0, - UnloadableDummyTypeSerializer.class, previousSerializersAndConfigs.get(0).f1, transactionSerializer); CompatibilityResult 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()))); - } } } 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 c5a68fb912a4a..be85748bd8625 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 @@ -137,14 +137,10 @@ public void startTimerService( // the following is the case where we restore if (restoredTimersSnapshot != null) { CompatibilityResult keySerializerCompatibility = CompatibilityUtil.resolveCompatibilityResult( - this.keyDeserializer, - null, restoredTimersSnapshot.getKeySerializerConfigSnapshot(), keySerializer); CompatibilityResult namespaceSerializerCompatibility = CompatibilityUtil.resolveCompatibilityResult( - this.namespaceDeserializer, - null, restoredTimersSnapshot.getNamespaceSerializerConfigSnapshot(), namespaceSerializer); 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 635851ed259c2..93366d47d41b2 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 @@ -22,10 +22,8 @@ import org.apache.flink.api.common.typeutils.CompatibilityResult; import org.apache.flink.api.common.typeutils.CompatibilityUtil; 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; @@ -298,17 +296,11 @@ public CompatibilityResult ensureCompatibility(TypeSerializerConf } CompatibilityResult 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(); } From c91d045c5eb6e355981e4edaa6d1a0d48e5d4a5e Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Thu, 14 Jun 2018 16:41:45 +0200 Subject: [PATCH 3/5] [FLINK-9377] [core] (part 3) Deprecate TypeSerializerSerializationUtil This commit deprecates all utility methods and classes related to serializing serializers. All methods that will still be in use, i.e. writing config snapshots, are now moved to a separate new TypeSerializerConfigSnapshotSerializationUtil class. --- ...alizerConfigSnapshotSerializationUtil.java | 185 ++++++++++++++++++ .../TypeSerializerSerializationUtil.java | 160 +-------------- .../typeutils/runtime/PojoSerializer.java | 13 +- .../common/typeutils/SerializerTestBase.java | 4 +- .../TypeSerializerSerializationUtilTest.java | 19 +- .../typeutils/base/EnumSerializerTest.java | 6 +- .../base/EnumSerializerUpgradeTest.java | 6 +- .../typeutils/runtime/PojoSerializerTest.java | 22 +-- .../kryo/KryoSerializerCompatibilityTest.java | 11 +- .../EnumValueSerializerUpgradeTest.scala | 6 +- 10 files changed, 237 insertions(+), 195 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotSerializationUtil.java 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 0000000000000..d7d4e16e45c8f --- /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. + * + *

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. + * + *

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> 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> readSerializerConfigSnapshots( + DataInputView in, + ClassLoader userCodeClassLoader) throws IOException { + + int numFields = in.readInt(); + final List> 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 serializerConfigSnapshotClass; + try { + serializerConfigSnapshotClass = (Class) + 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/TypeSerializerSerializationUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationUtil.java index e83b8c71ec35b..f6258836b33fb 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()); @@ -208,7 +213,7 @@ public static List, TypeSerializerConfigSnapshot>> read serializer = tryReadSerializer(bufferWrapper, userCodeClassLoader, true); bufferWithPos.setPosition(offsets[i * 2 + 1]); - configSnapshot = readSerializerConfigSnapshot(bufferWrapper, userCodeClassLoader); + configSnapshot = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(bufferWrapper, userCodeClassLoader); serializersAndConfigSnapshots.add( new Tuple2, TypeSerializerConfigSnapshot>(serializer, configSnapshot)); @@ -218,95 +223,6 @@ public static List, TypeSerializerConfigSnapshot>> read return serializersAndConfigSnapshots; } - /** - * Writes a {@link TypeSerializerConfigSnapshot} to the provided data output view. - * - *

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

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 +300,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 serializerConfigSnapshotClass; - try { - serializerConfigSnapshotClass = (Class) - 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/java/typeutils/runtime/PojoSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java index 39b91245b419d..f507f8a3a7a89 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 @@ -38,6 +38,7 @@ 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.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -802,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 @@ -819,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 @@ -836,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()); @@ -891,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, @@ -917,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, @@ -943,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/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 d32cb0cd34174..a2844d9cfe603 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 @@ -109,14 +109,14 @@ public void testSnapshotConfigurationAndReconfigure() throws Exception { byte[] serializedConfig; try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { - TypeSerializerSerializationUtil.writeSerializerConfigSnapshot( + TypeSerializerConfigSnapshotSerializationUtil.writeSerializerConfigSnapshot( new DataOutputViewStreamWrapper(out), configSnapshot); serializedConfig = out.toByteArray(); } TypeSerializerConfigSnapshot restoredConfig; try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { - restoredConfig = TypeSerializerSerializationUtil.readSerializerConfigSnapshot( + restoredConfig = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot( new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } 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 a7136fcd18bd2..7f1e0f61574ad 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 @@ -156,23 +156,22 @@ public void testSerializeConfigurationSnapshots() throws Exception { 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> 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( + 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) { 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 e3ce3ee6fca35..de564869d36a5 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 @@ -21,7 +21,7 @@ import org.apache.flink.api.common.typeutils.CompatibilityResult; 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; @@ -95,14 +95,14 @@ 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; try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { - restoredConfig = TypeSerializerSerializationUtil.readSerializerConfigSnapshot( + restoredConfig = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot( new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } 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 1f67acbab7d53..0701b15e15b43 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 @@ -20,7 +20,7 @@ import org.apache.flink.api.common.typeutils.CompatibilityResult; 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; @@ -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,7 +110,7 @@ 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)); 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 bdb3f8fb3e4d4..6b39a86bf05be 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 @@ -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; @@ -297,7 +297,7 @@ public void testReconfigureWithDifferentPojoType() throws Exception { TypeSerializerConfigSnapshot 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,7 +306,7 @@ 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()); } @@ -333,7 +333,7 @@ public void testReconfigureDifferentSubclassRegistrationOrder() throws Exception TypeSerializerConfigSnapshot 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,7 +346,7 @@ 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()); } @@ -379,7 +379,7 @@ public void testReconfigureRepopulateNonregisteredSubclassSerializerCache() thro TypeSerializerConfigSnapshot 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,7 +389,7 @@ 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()); } @@ -437,7 +437,7 @@ public void testReconfigureWithPreviouslyNonregisteredSubclasses() throws Except TypeSerializerConfigSnapshot 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,7 +449,7 @@ 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()); } @@ -552,7 +552,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,7 +567,7 @@ public void testSerializerSerializationFailureResilience() throws Exception{ PojoSerializer.PojoSerializerConfigSnapshot deserializedConfig; try(ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { deserializedConfig = (PojoSerializer.PojoSerializerConfigSnapshot) - TypeSerializerSerializationUtil.readSerializerConfigSnapshot( + TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot( new DataInputViewStreamWrapper(in), new ArtificialCNFExceptionThrowingClassLoader( Thread.currentThread().getContextClassLoader(), 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 89e9ec3f8c9e7..b1ff10d0155ab 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 @@ -22,6 +22,7 @@ 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.TypeSerializerConfigSnapshotSerializationUtil; import org.apache.flink.api.common.typeutils.TypeSerializerSerializationUtil; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; @@ -63,7 +64,7 @@ public void testMigrationStrategyForRemovedAvroDependency() throws Exception { // read configuration again from bytes TypeSerializerConfigSnapshot kryoSerializerConfigSnapshot; try (InputStream in = getClass().getResourceAsStream("/kryo-serializer-flink1.3-snapshot")) { - kryoSerializerConfigSnapshot = TypeSerializerSerializationUtil.readSerializerConfigSnapshot( + kryoSerializerConfigSnapshot = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot( new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } CompatibilityResult compatResult = kryoSerializerForA.ensureCompatibility(kryoSerializerConfigSnapshot); @@ -97,7 +98,7 @@ public void testMigrationStrategyWithDifferentKryoType() throws Exception { TypeSerializerConfigSnapshot 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,7 +106,7 @@ 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()); } @@ -251,7 +252,7 @@ public void testMigrationStrategyForDifferentRegistrationOrder() throws Exceptio TypeSerializerConfigSnapshot 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,7 +265,7 @@ 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()); } 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 8be4debff353f..1fed5e425142a 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.{CompatibilityResult, TypeSerializerConfigSnapshotSerializationUtil} import org.apache.flink.core.memory.{DataInputViewStreamWrapper, DataOutputViewStreamWrapper} import org.apache.flink.util.TestLogger import org.junit.rules.TemporaryFolder @@ -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,7 +144,7 @@ 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) From e09f91469fb6c86f5d2f05b78a9db3d9af8cce87 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 18 Jun 2018 16:24:08 +0200 Subject: [PATCH 4/5] [FLINK-9377] [core] (part 4) Introduce BackwardsCompatibleConfigSnapshot The BackwardsCompatibleConfigSnapshot is a wrapper, dummy config snapshot which wraps an actual config snapshot, as well as a pre-existing serializer instance. In previous versions, since the config snapshot wasn't a serializer factory but simply a container for serializer parameters, previous serializers didn't necessarily have config snapshots that are capable of correctly creating a correct corresponding restore serializer. In this case, since previous serializers still have serializers written in the checkpoint, the backwards compatible solution would be to wrap the written serializer and the config snapshot within the BackwardsCompatibleConfigSnapshot dummy. When attempting to restore the serializer, the wrapped serializer instance is returned instead of actually calling the restoreSerializer method of the wrapped config snapshot. --- .../BackwardsCompatibleConfigSnapshot.java | 102 ++++++++++++++++++ .../common/typeutils/CompatibilityUtil.java | 2 +- .../api/common/typeutils/TypeSerializer.java | 26 ++++- .../TypeSerializerSerializationUtil.java | 15 ++- .../common/typeutils/SerializerTestBase.java | 4 +- .../TypeSerializerSerializationUtilTest.java | 2 +- .../typeutils/base/EnumSerializerTest.java | 6 +- .../base/EnumSerializerUpgradeTest.java | 2 +- .../typeutils/runtime/PojoSerializerTest.java | 12 +-- .../kryo/KryoSerializerCompatibilityTest.java | 6 +- .../BackwardsCompatibleAvroSerializer.java | 6 +- ...BackwardsCompatibleAvroSerializerTest.java | 6 +- .../state/OperatorStateBackendTest.java | 2 +- .../TupleSerializerCompatibilityTest.scala | 3 +- .../typeutils/EnumValueSerializerTest.scala | 2 +- .../EnumValueSerializerUpgradeTest.scala | 2 +- 16 files changed, 166 insertions(+), 32 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/BackwardsCompatibleConfigSnapshot.java 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 0000000000000..f3cef6c9b0769 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/BackwardsCompatibleConfigSnapshot.java @@ -0,0 +1,102 @@ +/* + * 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. + * + *

If this placeholder config snapshot is provided to a new serializer for compatibility checks, the wrapped + * config snapshot will be "unwrapped" and be provided instead. + * + * @see TypeSerializer#internalEnsureCompatibility(TypeSerializerConfigSnapshot) + * + * @param the data type that the wrapped serializer instance serializes. + */ +@Internal +public class BackwardsCompatibleConfigSnapshot extends TypeSerializerConfigSnapshot { + + private TypeSerializerConfigSnapshot wrappedConfigSnapshot; + + private TypeSerializer serializerInstance; + + public BackwardsCompatibleConfigSnapshot( + TypeSerializerConfigSnapshot wrappedConfigSnapshot, + TypeSerializer 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 restoreSerializer() { + return serializerInstance; + } + + 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/CompatibilityUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompatibilityUtil.java index cff464943b934..0cfc03e6075da 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 @@ -55,7 +55,7 @@ public static CompatibilityResult resolveCompatibilityResult( TypeSerializer newSerializer) { if (precedingSerializerConfigSnapshot != null) { - return newSerializer.ensureCompatibility(precedingSerializerConfigSnapshot); + return newSerializer.internalEnsureCompatibility(precedingSerializerConfigSnapshot); } else { // if the configuration snapshot of the preceding serializer cannot be provided, // we can only simply assume that the new serializer is compatible 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 017e09ebb3811..8eab786387f96 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 @@ -18,6 +18,7 @@ package org.apache.flink.api.common.typeutils; +import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -215,5 +216,28 @@ public abstract class TypeSerializer implements Serializable { * * @return the determined compatibility result (cannot be {@code null}). */ - public abstract CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot); + protected abstract CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot); + + /** + * Public-facing method for serializer compatibility checks. Restored configuration snapshots should + * be provided via this method. + * + *

Before passing the configuration snapshot to the actual + * {@link #ensureCompatibility(TypeSerializerConfigSnapshot)} method, the configuration snapshot is checked + * to see if it is a dummy {@link BackwardsCompatibleConfigSnapshot}. If so, then the actual wrapped + * configuration snapshot is extracted and used instead. + * + * @param configSnapshot configuration snapshot of a preceding serializer for the same managed state + * + * @return the determined compatibility result (cannot be {@code null}). + */ + @Internal + public final CompatibilityResult internalEnsureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof BackwardsCompatibleConfigSnapshot) { + return ensureCompatibility( + ((BackwardsCompatibleConfigSnapshot) configSnapshot).getWrappedConfigSnapshot()); + } else { + return ensureCompatibility(configSnapshot); + } + } } 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 f6258836b33fb..d02a02c0ac6c2 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 @@ -202,7 +202,7 @@ public static List, TypeSerializerConfigSnapshot>> read new ArrayList<>(numSerializersAndConfigSnapshots); TypeSerializer serializer; - TypeSerializerConfigSnapshot configSnapshot; + TypeSerializerConfigSnapshot configSnapshot; try ( ByteArrayInputStreamWithPos bufferWithPos = new ByteArrayInputStreamWithPos(buffer); DataInputViewStreamWrapper bufferWrapper = new DataInputViewStreamWrapper(bufferWithPos)) { @@ -213,10 +213,17 @@ public static List, TypeSerializerConfigSnapshot>> read serializer = tryReadSerializer(bufferWrapper, userCodeClassLoader, true); bufferWithPos.setPosition(offsets[i * 2 + 1]); - configSnapshot = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(bufferWrapper, userCodeClassLoader); - serializersAndConfigSnapshots.add( - new Tuple2, 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)); } } 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 a2844d9cfe603..9aea9d52f61d0 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 @@ -120,11 +120,11 @@ public void testSnapshotConfigurationAndReconfigure() throws Exception { new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } - CompatibilityResult strategy = getSerializer().ensureCompatibility(restoredConfig); + CompatibilityResult strategy = getSerializer().internalEnsureCompatibility(restoredConfig); assertFalse(strategy.isRequiresMigration()); // also verify that the serializer's reconfigure implementation detects incompatibility - strategy = getSerializer().ensureCompatibility(new TestIncompatibleSerializerConfigSnapshot<>()); + strategy = getSerializer().internalEnsureCompatibility(new TestIncompatibleSerializerConfigSnapshot<>()); assertTrue(strategy.isRequiresMigration()); } 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 7f1e0f61574ad..da00fe7699f39 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 @@ -416,7 +416,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { @Override public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { - return IntSerializer.INSTANCE.ensureCompatibility(configSnapshot); + return IntSerializer.INSTANCE.internalEnsureCompatibility(configSnapshot); } @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 de564869d36a5..0001c9ac39026 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 @@ -72,7 +72,7 @@ public void testReconfiguration() { assertEquals(PublicEnum.PAULA.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.PAULA).intValue()); // reconfigure and verify compatibility - CompatibilityResult compatResult = serializer.ensureCompatibility( + CompatibilityResult compatResult = serializer.internalEnsureCompatibility( new EnumSerializer.EnumSerializerConfigSnapshot<>(PublicEnum.class, mockPreviousOrder)); assertFalse(compatResult.isRequiresMigration()); @@ -106,7 +106,7 @@ public void testConfigurationSnapshotSerialization() throws Exception { new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } - CompatibilityResult compatResult = serializer.ensureCompatibility(restoredConfig); + CompatibilityResult compatResult = serializer.internalEnsureCompatibility(restoredConfig); assertFalse(compatResult.isRequiresMigration()); assertEquals(PublicEnum.FOO.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.FOO).intValue()); @@ -161,7 +161,7 @@ public void testSerializeReconfiguredEnumSerializer() throws Exception { assertEquals(PublicEnum.PAULA.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.PAULA).intValue()); // reconfigure and verify compatibility - CompatibilityResult compatResult = serializer.ensureCompatibility( + CompatibilityResult compatResult = serializer.internalEnsureCompatibility( new EnumSerializer.EnumSerializerConfigSnapshot<>(PublicEnum.class, mockPreviousOrder)); assertFalse(compatResult.isRequiresMigration()); 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 0701b15e15b43..09aeeb8ca590f 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 @@ -114,7 +114,7 @@ private static CompatibilityResult checkCompatibility(String enumSourceA, String } EnumSerializer enumSerializer2 = new EnumSerializer(classLoader2.loadClass(ENUM_NAME)); - return enumSerializer2.ensureCompatibility(restoredSnapshot); + return enumSerializer2.internalEnsureCompatibility(restoredSnapshot); } 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 6b39a86bf05be..09e3c26ca8055 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 @@ -310,7 +310,7 @@ public void testReconfigureWithDifferentPojoType() throws Exception { new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } - CompatibilityResult compatResult = pojoSerializer2.ensureCompatibility(pojoSerializerConfigSnapshot); + CompatibilityResult compatResult = pojoSerializer2.internalEnsureCompatibility(pojoSerializerConfigSnapshot); assertTrue(compatResult.isRequiresMigration()); } @@ -350,7 +350,7 @@ public void testReconfigureDifferentSubclassRegistrationOrder() throws Exception new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } - CompatibilityResult compatResult = pojoSerializer.ensureCompatibility(pojoSerializerConfigSnapshot); + CompatibilityResult compatResult = pojoSerializer.internalEnsureCompatibility(pojoSerializerConfigSnapshot); assertTrue(!compatResult.isRequiresMigration()); // reconfigure - check reconfiguration result and that registration ids remains the same @@ -394,7 +394,7 @@ public void testReconfigureRepopulateNonregisteredSubclassSerializerCache() thro } // reconfigure - check reconfiguration result and that subclass serializer cache is repopulated - CompatibilityResult compatResult = pojoSerializer.ensureCompatibility(pojoSerializerConfigSnapshot); + CompatibilityResult compatResult = pojoSerializer.internalEnsureCompatibility(pojoSerializerConfigSnapshot); assertFalse(compatResult.isRequiresMigration()); assertEquals(2, pojoSerializer.getSubclassSerializerCache().size()); assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassA.class)); @@ -456,7 +456,7 @@ public void testReconfigureWithPreviouslyNonregisteredSubclasses() throws Except // reconfigure - check reconfiguration result and that // 1) subclass serializer cache is repopulated // 2) registrations also contain the now registered subclasses - CompatibilityResult compatResult = pojoSerializer.ensureCompatibility(pojoSerializerConfigSnapshot); + CompatibilityResult compatResult = pojoSerializer.internalEnsureCompatibility(pojoSerializerConfigSnapshot); assertFalse(compatResult.isRequiresMigration()); assertEquals(2, pojoSerializer.getSubclassSerializerCache().size()); assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassA.class)); @@ -533,7 +533,7 @@ public void testReconfigureWithDifferentFieldOrder() throws Exception { new HashMap, Tuple2, TypeSerializerConfigSnapshot>>()); // empty; irrelevant for this test // reconfigure - check reconfiguration result and that fields are reordered to the previous order - CompatibilityResult compatResult = pojoSerializer.ensureCompatibility(mockPreviousConfigSnapshot); + CompatibilityResult compatResult = pojoSerializer.internalEnsureCompatibility(mockPreviousConfigSnapshot); assertFalse(compatResult.isRequiresMigration()); int i = 0; for (Field field : mockOriginalFieldOrder) { @@ -574,7 +574,7 @@ public void testSerializerSerializationFailureResilience() throws Exception{ cnfThrowingClassnames)); } - Assert.assertFalse(pojoSerializer.ensureCompatibility(deserializedConfig).isRequiresMigration()); + Assert.assertFalse(pojoSerializer.internalEnsureCompatibility(deserializedConfig).isRequiresMigration()); 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 b1ff10d0155ab..222c32a8d860e 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 @@ -67,7 +67,7 @@ public void testMigrationStrategyForRemovedAvroDependency() throws Exception { kryoSerializerConfigSnapshot = TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot( new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } - CompatibilityResult compatResult = kryoSerializerForA.ensureCompatibility(kryoSerializerConfigSnapshot); + CompatibilityResult compatResult = kryoSerializerForA.internalEnsureCompatibility(kryoSerializerConfigSnapshot); assertFalse(compatResult.isRequiresMigration()); } @@ -110,7 +110,7 @@ public void testMigrationStrategyWithDifferentKryoType() throws Exception { new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } - CompatibilityResult compatResult = kryoSerializerForB.ensureCompatibility(kryoSerializerConfigSnapshot); + CompatibilityResult compatResult = kryoSerializerForB.internalEnsureCompatibility(kryoSerializerConfigSnapshot); assertTrue(compatResult.isRequiresMigration()); } @@ -270,7 +270,7 @@ public void testMigrationStrategyForDifferentRegistrationOrder() throws Exceptio } // reconfigure - check reconfiguration result and that registration id remains the same - CompatibilityResult compatResult = kryoSerializer.ensureCompatibility(kryoSerializerConfigSnapshot); + CompatibilityResult compatResult = kryoSerializer.internalEnsureCompatibility(kryoSerializerConfigSnapshot); assertFalse(compatResult.isRequiresMigration()); assertEquals(testClassId, kryoSerializer.getKryo().getRegistration(TestClass.class).getId()); assertEquals(testClassAId, kryoSerializer.getKryo().getRegistration(TestClassA.class).getId()); 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 addd4b1295423..573f6940cae19 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 @@ -186,7 +186,7 @@ public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot pojoSerializer = (TypeSerializer) typeInfo.createPojoSerializer(new ExecutionConfig()); this.serializer = pojoSerializer; - return serializer.ensureCompatibility(configSnapshot); + return serializer.internalEnsureCompatibility(configSnapshot); } 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 serializer.internalEnsureCompatibility(configSnapshot); } else { // completely incompatible type, needs migration 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 f64163656fa78..29a8adce94a6f 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 @@ -102,10 +102,10 @@ 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(serializer.internalEnsureCompatibility(configSnapshot).isRequiresMigration()); final TypeSerializer newSerializer = new AvroTypeInfo<>(User.class, true).createSerializer(new ExecutionConfig()); - assertFalse(newSerializer.ensureCompatibility(configSnapshot).isRequiresMigration()); + assertFalse(newSerializer.internalEnsureCompatibility(configSnapshot).isRequiresMigration()); // deserialize the data and make sure this still works validateDeserialization(newSerializer); @@ -113,7 +113,7 @@ public void testCompatibilityWithPojoSerializer() throws Exception { TypeSerializerConfigSnapshot nextSnapshot = newSerializer.snapshotConfiguration(); final TypeSerializer nextSerializer = new AvroTypeInfo<>(User.class, true).createSerializer(new ExecutionConfig()); - assertFalse(nextSerializer.ensureCompatibility(nextSnapshot).isRequiresMigration()); + assertFalse(nextSerializer.internalEnsureCompatibility(nextSnapshot).isRequiresMigration()); // deserialize the data and make sure this still works validateDeserialization(nextSerializer); 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 bbf7fd5676382..114a8304a0c5c 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 @@ -357,7 +357,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { @Override public CompatibilityResult ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { - return IntSerializer.INSTANCE.ensureCompatibility(configSnapshot); + return IntSerializer.INSTANCE.internalEnsureCompatibility(configSnapshot); } } 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 7d594fb2b16c3..29991dba9f4a5 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(currentSerializer + .internalEnsureCompatibility(oldConfigSnapshot).isRequiresMigration) // 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 f89e9bc473860..be7cfd48413ca 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(enumSerializer.internalEnsureCompatibility(snapshot).isRequiresMigration) } } 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 1fed5e425142a..ff2b1247b747e 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 @@ -150,7 +150,7 @@ class EnumValueSerializerUpgradeTest extends TestLogger with JUnitSuiteLike { val enum2 = instantiateEnum[Enumeration](classLoader2, enumName) val enumValueSerializer2 = new EnumValueSerializer(enum2) - enumValueSerializer2.ensureCompatibility(snapshot2) + enumValueSerializer2.internalEnsureCompatibility(snapshot2) } } From da84665a9b101a803f7446210afc34bbd4a71703 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 2 Jul 2018 11:45:20 +0800 Subject: [PATCH 5/5] [FLINK-9377] [core] (part 5) Remove serializers from checkpoint state meta infos This commit officially removes the behaviour of writing serializers in the state meta info of keyed state, operator state, and timers state. This affects the serialization formats of the KeyedBackendSerializationProxy, OperatorBackendSerializationProxy, and InternalTimerServiceSerializationProxy, and therefore their versions are all upticked. --- .../state/DefaultOperatorStateBackend.java | 23 ++- .../state/KeyedBackendSerializationProxy.java | 58 +++--- ...endStateMetaInfoSnapshotReaderWriters.java | 109 +++++++++--- .../OperatorBackendSerializationProxy.java | 6 +- ...endStateMetaInfoSnapshotReaderWriters.java | 165 ++++++++++++++---- ...gisteredBroadcastBackendStateMetaInfo.java | 26 +-- .../RegisteredKeyedBackendStateMetaInfo.java | 53 ++---- ...egisteredOperatorBackendStateMetaInfo.java | 13 +- .../state/heap/HeapKeyedStateBackend.java | 4 +- .../state/SerializationProxiesTest.java | 13 -- .../state/RocksDBKeyedStateBackend.java | 12 +- .../operators/HeapInternalTimerService.java | 22 ++- ...nternalTimerServiceSerializationProxy.java | 7 +- .../api/operators/InternalTimersSnapshot.java | 47 ++--- .../InternalTimersSnapshotReaderWriters.java | 95 +++++++--- 15 files changed, 405 insertions(+), 248 deletions(-) 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 58083b8c87612..e0b40c86a28d2 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 @@ -499,8 +499,10 @@ public void restore(Collection 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 @@ -521,7 +523,7 @@ public void restore(Collection restoreSnapshots) throws Exc listState = new PartitionableListState<>( new RegisteredOperatorBackendStateMetaInfo<>( restoredMetaInfo.getName(), - restoredMetaInfo.getPartitionStateSerializer(), + restoredPartitionStateSerializer, restoredMetaInfo.getAssignmentMode())); registeredOperatorStates.put(listState.getStateMetaInfo().getName(), listState); @@ -536,9 +538,14 @@ public void restore(Collection 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 @@ -560,8 +567,8 @@ public void restore(Collection restoreSnapshots) throws Exc new RegisteredBroadcastBackendStateMetaInfo<>( restoredMetaInfo.getName(), restoredMetaInfo.getAssignmentMode(), - restoredMetaInfo.getKeySerializer(), - restoredMetaInfo.getValueSerializer())); + restoredKeySerializer, + restoredValueSerializer)); registeredBroadcastStates.put(broadcastState.getStateMetaInfo().getName(), broadcastState); } else { 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 f32e646e6efd9..10d681e3f2f70 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 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 @@ public class KeyedBackendSerializationProxy extends VersionedIOReadableWritab /** This specifies whether or not to use dummy {@link UnloadableDummyTypeSerializer} when serializers cannot be read. */ private boolean isSerializerPresenceRequired; - private TypeSerializer keySerializer; - private TypeSerializerConfigSnapshot keySerializerConfigSnapshot; + private TypeSerializerConfigSnapshot keySerializerConfigSnapshot; private List> 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 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, 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, TypeSerializerConfigSnapshot> keySerializerAndConfig = - TypeSerializerSerializationUtil.readSerializersAndConfigsWithResilience(in, userCodeClassLoader).get(0); - this.keySerializer = (TypeSerializer) keySerializerAndConfig.f0; - this.keySerializerConfigSnapshot = keySerializerAndConfig.f1; + TypeSerializerSerializationUtil.readSerializersAndConfigsWithResilience(in, userCodeClassLoader).get(0); + + this.keySerializerConfigSnapshot = new BackwardsCompatibleConfigSnapshot( + keySerializerAndConfig.f1, + (TypeSerializer) 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 f99234180c90b..5277de7fdcfc9 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 @@ public class KeyedBackendStateMetaInfoSnapshotReaderWriters { // 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 KeyedBackendStateMetaInfoWriter getWriterForVersion( @@ -52,9 +57,12 @@ public static KeyedBackendStateMetaInfoWriter getWriterForVersion( 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 extends AbstractKeyedBackendStateMetaInfoWriter { + static class KeyedBackendStateMetaInfoWriterV3V4 extends AbstractKeyedBackendStateMetaInfoWriter { - public KeyedBackendStateMetaInfoWriterV3(RegisteredKeyedBackendStateMetaInfo.Snapshot stateMetaInfo) { + public KeyedBackendStateMetaInfoWriterV3V4(RegisteredKeyedBackendStateMetaInfo.Snapshot 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, TypeSerializerConfigSnapshot>( - stateMetaInfo.getNamespaceSerializer(), stateMetaInfo.getNamespaceSerializerConfigSnapshot()), - new Tuple2, TypeSerializerConfigSnapshot>( - stateMetaInfo.getStateSerializer(), stateMetaInfo.getStateSerializerConfigSnapshot()))); + new Tuple2<>( + null, stateMetaInfo.getNamespaceSerializerConfigSnapshot()), + new Tuple2<>( + null, stateMetaInfo.getStateSerializerConfigSnapshot()))); + } + } + + static class KeyedBackendStateMetaInfoWriterV5 extends AbstractKeyedBackendStateMetaInfoWriter { + + public KeyedBackendStateMetaInfoWriterV5(RegisteredKeyedBackendStateMetaInfo.Snapshot 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 RegisteredKeyedBackendStateMetaInfo.Snapshot readStateMetaInfo(Data metaInfo.setStateType(StateDescriptor.Type.values()[in.readInt()]); metaInfo.setName(in.readUTF()); - metaInfo.setNamespaceSerializer(TypeSerializerSerializationUtil.tryReadSerializer(in, userCodeClassLoader, true)); - metaInfo.setStateSerializer(TypeSerializerSerializationUtil.tryReadSerializer(in, userCodeClassLoader, true)); + final TypeSerializerSerializationUtil.TypeSerializerSerializationProxy 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 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 extends AbstractKeyedBackendStateMetaInfoReader { + static class KeyedBackendStateMetaInfoReaderV3V4 extends AbstractKeyedBackendStateMetaInfoReader { - public KeyedBackendStateMetaInfoReaderV3(ClassLoader userCodeClassLoader) { + public KeyedBackendStateMetaInfoReaderV3V4(ClassLoader userCodeClassLoader) { super(userCodeClassLoader); } @@ -199,11 +241,34 @@ public RegisteredKeyedBackendStateMetaInfo.Snapshot readStateMetaInfo(Data List, TypeSerializerConfigSnapshot>> serializersAndConfigs = TypeSerializerSerializationUtil.readSerializersAndConfigsWithResilience(in, userCodeClassLoader); - metaInfo.setNamespaceSerializer((TypeSerializer) 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 extends AbstractKeyedBackendStateMetaInfoReader { + + public KeyedBackendStateMetaInfoReaderV5(ClassLoader userCodeClassLoader) { + super(userCodeClassLoader); + } + + @SuppressWarnings("unchecked") + @Override + public RegisteredKeyedBackendStateMetaInfo.Snapshot readStateMetaInfo(DataInputView in) throws IOException { + RegisteredKeyedBackendStateMetaInfo.Snapshot metaInfo = + new RegisteredKeyedBackendStateMetaInfo.Snapshot<>(); + + metaInfo.setStateType(StateDescriptor.Type.values()[in.readInt()]); + metaInfo.setName(in.readUTF()); - metaInfo.setStateSerializer((TypeSerializer) 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 e73f83a9e1d96..d1767d50e8e4d 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> operatorStateMetaInfoSnapshots; private List> 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 fafd5423e01c3..9a0329298de2d 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 OperatorBackendStateMetaInfoWriter getOperatorStateWriterForVersion( @@ -59,9 +58,12 @@ public static OperatorBackendStateMetaInfoWriter getOperatorStateWriterForVe // 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 @@ public static BroadcastStateMetaInfoWriter getBroadcastStateWriterForVers final RegisteredBroadcastBackendStateMetaInfo.Snapshot 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 extends AbstractOperatorBackendStateMetaInfoWriter { + public static class OperatorBackendStateMetaInfoWriterV2V3 extends AbstractOperatorBackendStateMetaInfoWriter { - public OperatorBackendStateMetaInfoWriterV2(RegisteredOperatorBackendStateMetaInfo.Snapshot stateMetaInfo) { + public OperatorBackendStateMetaInfoWriterV2V3(RegisteredOperatorBackendStateMetaInfo.Snapshot 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, 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 extends AbstractOperatorBackendStateMetaInfoWriter { + + public OperatorBackendStateMetaInfoWriterV4(RegisteredOperatorBackendStateMetaInfo.Snapshot 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 extends AbstractBroadcastStateMetaInfoWriter { 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 extends AbstractBroadcastStateMetaInfoWriter { + + public BroadcastStateMetaInfoWriterV4( + final RegisteredBroadcastBackendStateMetaInfo.Snapshot 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 OperatorBackendStateMetaInfoReader getOperatorStateReaderForVersion( @@ -190,10 +242,13 @@ public static OperatorBackendStateMetaInfoReader getOperatorStateReaderFo 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 static BroadcastStateMetaInfoReader getBroadcastStateReaderF 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 RegisteredOperatorBackendStateMetaInfo.Snapshot readOperatorStateMetaI DataInputViewStream dis = new DataInputViewStream(in); ClassLoader previousClassLoader = Thread.currentThread().getContextClassLoader(); + TypeSerializer stateSerializer; try ( InstantiationUtil.FailureTolerantObjectInputStream ois = new InstantiationUtil.FailureTolerantObjectInputStream(dis, userCodeClassLoader)) { Thread.currentThread().setContextClassLoader(userCodeClassLoader); - TypeSerializer stateSerializer = (TypeSerializer) ois.readObject(); - stateMetaInfo.setPartitionStateSerializer(stateSerializer); + stateSerializer = (TypeSerializer) 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 extends AbstractOperatorBackendStateMetaInfoReader { + public static class OperatorBackendStateMetaInfoReaderV2V3 extends AbstractOperatorBackendStateMetaInfoReader { - public OperatorBackendStateMetaInfoReaderV2(ClassLoader userCodeClassLoader) { + public OperatorBackendStateMetaInfoReaderV2V3(ClassLoader userCodeClassLoader) { super(userCodeClassLoader); } @@ -300,13 +359,35 @@ public RegisteredOperatorBackendStateMetaInfo.Snapshot readOperatorStateMetaI Tuple2, TypeSerializerConfigSnapshot> stateSerializerAndConfig = TypeSerializerSerializationUtil.readSerializersAndConfigsWithResilience(in, userCodeClassLoader).get(0); - stateMetaInfo.setPartitionStateSerializer((TypeSerializer) stateSerializerAndConfig.f0); - stateMetaInfo.setPartitionStateSerializerConfigSnapshot(stateSerializerAndConfig.f1); + stateMetaInfo.setPartitionStateSerializerConfigSnapshot( + new BackwardsCompatibleConfigSnapshot(stateSerializerAndConfig.f1, stateSerializerAndConfig.f0)); + + return stateMetaInfo; + } + } + + public static class OperatorBackendStateMetaInfoReaderV4 extends AbstractOperatorBackendStateMetaInfoReader { + + public OperatorBackendStateMetaInfoReaderV4(ClassLoader userCodeClassLoader) { + super(userCodeClassLoader); + } + + @Override + public RegisteredOperatorBackendStateMetaInfo.Snapshot readOperatorStateMetaInfo(DataInputView in) throws IOException { + RegisteredOperatorBackendStateMetaInfo.Snapshot 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 extends AbstractBroadcastStateMetaInfoReader { public BroadcastStateMetaInfoReaderV3(final ClassLoader userCodeClassLoader) { @@ -327,11 +408,35 @@ public RegisteredBroadcastBackendStateMetaInfo.Snapshot readBroadcastState Tuple2, TypeSerializerConfigSnapshot> keySerializerAndConfig = serializers.get(0); Tuple2, TypeSerializerConfigSnapshot> valueSerializerAndConfig = serializers.get(1); - stateMetaInfo.setKeySerializer((TypeSerializer) keySerializerAndConfig.f0); - stateMetaInfo.setKeySerializerConfigSnapshot(keySerializerAndConfig.f1); - - stateMetaInfo.setValueSerializer((TypeSerializer) 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 extends AbstractBroadcastStateMetaInfoReader { + + public BroadcastStateMetaInfoReaderV4(final ClassLoader userCodeClassLoader) { + super(userCodeClassLoader); + } + + @Override + public RegisteredBroadcastBackendStateMetaInfo.Snapshot readBroadcastStateMetaInfo(DataInputView in) throws IOException { + RegisteredBroadcastBackendStateMetaInfo.Snapshot 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 7204cd3856346..a491559840c7a 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 static class Snapshot { private String name; private OperatorStateHandle.Mode assignmentMode; - private TypeSerializer keySerializer; - private TypeSerializer 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 getKeySerializer() { - return keySerializer; - } - - void setKeySerializer(TypeSerializer serializer) { - this.keySerializer = serializer; - } - - public TypeSerializer getValueSerializer() { - return valueSerializer; - } - - void setValueSerializer(TypeSerializer 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 3f7a8d6f8247e..fbf620b60cd42 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 @@ -77,9 +77,7 @@ public Snapshot snapshot() { stateType, name, namespaceSerializer.duplicate(), - stateSerializer.duplicate(), - namespaceSerializer.snapshotConfiguration(), - stateSerializer.snapshotConfiguration()); + stateSerializer.duplicate()); } @Override @@ -132,10 +130,8 @@ public static class Snapshot { private StateDescriptor.Type stateType; private String name; - private TypeSerializer namespaceSerializer; - private TypeSerializer stateSerializer; - private TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot; - private TypeSerializerConfigSnapshot stateSerializerConfigSnapshot; + private TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot; + private TypeSerializerConfigSnapshot stateSerializerConfigSnapshot; /** Empty constructor used when restoring the state meta info snapshot. */ Snapshot() {} @@ -144,16 +140,15 @@ private Snapshot( StateDescriptor.Type stateType, String name, TypeSerializer namespaceSerializer, - TypeSerializer stateSerializer, - TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot, - TypeSerializerConfigSnapshot stateSerializerConfigSnapshot) { + TypeSerializer 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() { @@ -172,35 +167,19 @@ void setName(String name) { this.name = name; } - public TypeSerializer getNamespaceSerializer() { - return namespaceSerializer; - } - - void setNamespaceSerializer(TypeSerializer namespaceSerializer) { - this.namespaceSerializer = namespaceSerializer; - } - - public TypeSerializer getStateSerializer() { - return stateSerializer; - } - - void setStateSerializer(TypeSerializer stateSerializer) { - this.stateSerializer = stateSerializer; - } - - public TypeSerializerConfigSnapshot getNamespaceSerializerConfigSnapshot() { + public TypeSerializerConfigSnapshot getNamespaceSerializerConfigSnapshot() { return namespaceSerializerConfigSnapshot; } - void setNamespaceSerializerConfigSnapshot(TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot) { + void setNamespaceSerializerConfigSnapshot(TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot) { this.namespaceSerializerConfigSnapshot = namespaceSerializerConfigSnapshot; } - public TypeSerializerConfigSnapshot getStateSerializerConfigSnapshot() { + public TypeSerializerConfigSnapshot getStateSerializerConfigSnapshot() { return stateSerializerConfigSnapshot; } - void setStateSerializerConfigSnapshot(TypeSerializerConfigSnapshot stateSerializerConfigSnapshot) { + void setStateSerializerConfigSnapshot(TypeSerializerConfigSnapshot stateSerializerConfigSnapshot) { this.stateSerializerConfigSnapshot = stateSerializerConfigSnapshot; } @@ -225,9 +204,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()); } @@ -236,8 +213,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 + 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; 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 a9adc8d1e3a01..9d8428f8a3b2e 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 static class Snapshot { private String name; private OperatorStateHandle.Mode assignmentMode; - private TypeSerializer 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 getPartitionStateSerializer() { - return partitionStateSerializer; - } - - void setPartitionStateSerializer(TypeSerializer 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/heap/HeapKeyedStateBackend.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/HeapKeyedStateBackend.java index 87c3b57df541c..491e8cfc807ad 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 @@ -318,8 +318,8 @@ private void restorePartitionedState(Collection 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/test/java/org/apache/flink/runtime/state/SerializationProxiesTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/SerializationProxiesTest.java index 3f78097f11cd0..30dad1a3aa4b3 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-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 a7b8b4cb4580d..1d00267892f9b 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 @@ -643,8 +643,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); @@ -949,8 +949,8 @@ private ColumnFamilyHandle getOrRegisterColumnFamilyHandle( new RegisteredKeyedBackendStateMetaInfo<>( stateMetaInfoSnapshot.getStateType(), stateMetaInfoSnapshot.getName(), - stateMetaInfoSnapshot.getNamespaceSerializer(), - stateMetaInfoSnapshot.getStateSerializer()); + stateMetaInfoSnapshot.getNamespaceSerializerConfigSnapshot().restoreSerializer(), + stateMetaInfoSnapshot.getStateSerializerConfigSnapshot().restoreSerializer()); registeredStateMetaInfoEntry = new Tuple2<>( @@ -1082,8 +1082,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(), 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 be85748bd8625..5d447e04f2c47 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 @@ -253,9 +253,7 @@ public void advanceWatermark(long time) throws Exception { public InternalTimersSnapshot snapshotTimersForKeyGroup(int keyGroupIdx) { return new InternalTimersSnapshot<>( keySerializer, - keySerializer.snapshotConfiguration(), namespaceSerializer, - namespaceSerializer.snapshotConfiguration(), eventTimeTimersQueue.getTimersForKeyGroup(keyGroupIdx), processingTimeTimersQueue.getTimersForKeyGroup(keyGroupIdx)); } @@ -271,13 +269,18 @@ public InternalTimersSnapshot snapshotTimersForKeyGroup(int keyGroupIdx) { public void restoreTimersForKeyGroup(InternalTimersSnapshot restoredSnapshot, int keyGroupIdx) { this.restoredTimersSnapshot = (InternalTimersSnapshot) restoredSnapshot; - if (areSnapshotSerializersIncompatible(restoredSnapshot)) { + TypeSerializer restoredKeyDeserializer = + restoredTimersSnapshot.getKeySerializerConfigSnapshot().restoreSerializer(); + TypeSerializer 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."); @@ -332,8 +335,11 @@ List>> getProcessingTimeTimersPerKeyGroup() { 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 restoredKeySerializer, + TypeSerializer 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 efa93d3e266a0..900a81cc0d24f 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 extends PostVersionedIOReadableWritable { - public static final int VERSION = 1; + public static final int VERSION = 2; /** The key-group timer services to write / read. */ private Map> 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 ddaee6b2e7a44..cb2e0a163233c 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 { - private TypeSerializer keySerializer; - private TypeSerializerConfigSnapshot keySerializerConfigSnapshot; - private TypeSerializer namespaceSerializer; - private TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot; + private TypeSerializerConfigSnapshot keySerializerConfigSnapshot; + private TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot; private Set> eventTimeTimers; private Set> processingTimeTimers; @@ -46,49 +44,31 @@ public InternalTimersSnapshot() {} /** Constructor to use when snapshotting the timers. */ public InternalTimersSnapshot( TypeSerializer keySerializer, - TypeSerializerConfigSnapshot keySerializerConfigSnapshot, TypeSerializer namespaceSerializer, - TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot, @Nullable Set> eventTimeTimers, @Nullable Set> 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 getKeySerializer() { - return keySerializer; - } - - public void setKeySerializer(TypeSerializer keySerializer) { - this.keySerializer = keySerializer; - } - public TypeSerializerConfigSnapshot getKeySerializerConfigSnapshot() { return keySerializerConfigSnapshot; } - public void setKeySerializerConfigSnapshot(TypeSerializerConfigSnapshot keySerializerConfigSnapshot) { + public void setKeySerializerConfigSnapshot(TypeSerializerConfigSnapshot keySerializerConfigSnapshot) { this.keySerializerConfigSnapshot = keySerializerConfigSnapshot; } - public TypeSerializer getNamespaceSerializer() { - return namespaceSerializer; - } - - public void setNamespaceSerializer(TypeSerializer namespaceSerializer) { - this.namespaceSerializer = namespaceSerializer; - } - public TypeSerializerConfigSnapshot getNamespaceSerializerConfigSnapshot() { return namespaceSerializerConfigSnapshot; } - public void setNamespaceSerializerConfigSnapshot(TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot) { + public void setNamespaceSerializerConfigSnapshot(TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot) { this.namespaceSerializerConfigSnapshot = namespaceSerializerConfigSnapshot; } @@ -108,6 +88,17 @@ public void setProcessingTimeTimers(Set> processingTimeTimer this.processingTimeTimers = processingTimeTimers; } + public TimerHeapInternalTimer.TimerSerializer 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 05b77a7143cb0..b3c1c3359ce35 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 @@ public class InternalTimersSnapshotReaderWriters { // ------------------------------------------------------------------------------- // 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 InternalTimersSnapshotWriter getWriterForVersion(int version, InternalTimersSnapshot timersSnapshot) { @@ -58,9 +61,13 @@ public static InternalTimersSnapshotWriter getWriterForVersion(int versio 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 timersS public final void writeTimersSnapshot(DataOutputView out) throws IOException { writeKeyAndNamespaceSerializers(out); - TimerHeapInternalTimer.TimerSerializer timerSerializer = new TimerHeapInternalTimer.TimerSerializer<>( - timersSnapshot.getKeySerializer(), - timersSnapshot.getNamespaceSerializer()); + TimerHeapInternalTimer.TimerSerializer timerSerializer = timersSnapshot.createTimerSerializer(); // write the event time timers Set> eventTimers = timersSnapshot.getEventTimeTimers(); @@ -134,8 +139,10 @@ public InternalTimersSnapshotWriterPreVersioned(InternalTimersSnapshot 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 extends AbstractInternalTimersSnapshotWriter { + + public InternalTimersSnapshotWriterV2(InternalTimersSnapshot 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 InternalTimersSnapshotReader getReaderForVersion( @@ -172,9 +197,13 @@ public static InternalTimersSnapshotReader getReaderForVersion( 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 @@ public final InternalTimersSnapshot readTimersSnapshot(DataInputView in) t restoreKeyAndNamespaceSerializers(restoredTimersSnapshot, in); - TimerHeapInternalTimer.TimerSerializer timerSerializer = - new TimerHeapInternalTimer.TimerSerializer<>( - restoredTimersSnapshot.getKeySerializer(), - restoredTimersSnapshot.getNamespaceSerializer()); + TimerHeapInternalTimer.TimerSerializer 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 keySerializer = InstantiationUtil.deserializeObject(dis, userCodeClassLoader, true); + restoredTimersSnapshot.setKeySerializerConfigSnapshot(keySerializer.snapshotConfiguration()); + + TypeSerializer 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, TypeSerializerConfigSnapshot>> serializersAndConfigs = TypeSerializerSerializationUtil.readSerializersAndConfigsWithResilience(in, userCodeClassLoader); - restoredTimersSnapshot.setKeySerializer((TypeSerializer) serializersAndConfigs.get(0).f0); - restoredTimersSnapshot.setKeySerializerConfigSnapshot(serializersAndConfigs.get(0).f1); - restoredTimersSnapshot.setNamespaceSerializer((TypeSerializer) 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 extends AbstractInternalTimersSnapshotReader { + + public InternalTimersSnapshotReaderV2(ClassLoader userCodeClassLoader) { + super(userCodeClassLoader); + } + + @SuppressWarnings("unchecked") + @Override + protected void restoreKeyAndNamespaceSerializers( + InternalTimersSnapshot restoredTimersSnapshot, + DataInputView in) throws IOException { + + restoredTimersSnapshot.setKeySerializerConfigSnapshot( + TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(in, userCodeClassLoader)); + + restoredTimersSnapshot.setNamespaceSerializerConfigSnapshot( + TypeSerializerConfigSnapshotSerializationUtil.readSerializerConfigSnapshot(in, userCodeClassLoader)); } } }