From 538a7acecce0d72e36e3726c0df2b6b96be35fc3 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Mon, 1 May 2017 21:32:10 +0800 Subject: [PATCH 1/5] [FLINK-6190] [core] Migratable TypeSerializers This commit introduces the user-facing APIs for migratable TypeSerializers. The new user-facing APIs are: - new class: TypeSerializerConfigSnapshot - new class: ForwardCompatibleSerializationFormatConfig - new method: TypeSerializer#snapshotConfiguration() - new method: TypeSerializer#reconfigure(TypeSerializerConfigSnapshot) - new enum: ReconfigureResult --- .../typeutils/runtime/WritableSerializer.java | 45 +- ...CompositeTypeSerializerConfigSnapshot.java | 85 +++ ...rdCompatibleSerializationFormatConfig.java | 76 ++ .../GenericTypeSerializerConfigSnapshot.java | 88 +++ .../common/typeutils/MigrationStrategy.java | 87 +++ .../ParameterlessTypeSerializerConfig.java | 89 +++ .../api/common/typeutils/TypeSerializer.java | 91 +++ .../TypeSerializerConfigSnapshot.java | 103 +++ .../TypeSerializerSerializationProxy.java | 10 + .../common/typeutils/TypeSerializerUtil.java | 216 ++++++ .../typeutils/base/BooleanSerializer.java | 6 + .../base/BooleanValueSerializer.java | 9 +- .../common/typeutils/base/ByteSerializer.java | 7 +- .../typeutils/base/ByteValueSerializer.java | 9 +- .../common/typeutils/base/CharSerializer.java | 7 +- .../typeutils/base/CharValueSerializer.java | 9 +- .../CollectionSerializerConfigSnapshot.java | 44 ++ .../common/typeutils/base/DateSerializer.java | 7 + .../typeutils/base/DoubleSerializer.java | 7 +- .../typeutils/base/DoubleValueSerializer.java | 9 +- .../common/typeutils/base/EnumSerializer.java | 175 ++++- .../typeutils/base/FloatSerializer.java | 7 +- .../typeutils/base/FloatValueSerializer.java | 9 +- .../base/GenericArraySerializer.java | 40 +- .../GenericArraySerializerConfigSnapshot.java | 95 +++ .../common/typeutils/base/IntSerializer.java | 7 +- .../typeutils/base/IntValueSerializer.java | 9 +- .../common/typeutils/base/ListSerializer.java | 34 +- .../common/typeutils/base/LongSerializer.java | 7 +- .../typeutils/base/LongValueSerializer.java | 9 +- .../common/typeutils/base/MapSerializer.java | 43 +- .../base/MapSerializerConfigSnapshot.java | 48 ++ .../typeutils/base/ShortSerializer.java | 7 +- .../typeutils/base/ShortValueSerializer.java | 9 +- .../typeutils/base/SqlDateSerializer.java | 7 + .../typeutils/base/SqlTimeSerializer.java | 12 + .../typeutils/base/StringSerializer.java | 6 + .../typeutils/base/StringValueSerializer.java | 9 +- .../base/TypeSerializerSingleton.java | 32 + .../BooleanPrimitiveArraySerializer.java | 3 +- .../array/BytePrimitiveArraySerializer.java | 2 +- .../array/CharPrimitiveArraySerializer.java | 3 +- .../array/DoublePrimitiveArraySerializer.java | 3 +- .../array/FloatPrimitiveArraySerializer.java | 3 +- .../array/IntPrimitiveArraySerializer.java | 3 +- .../array/LongPrimitiveArraySerializer.java | 3 +- .../array/ShortPrimitiveArraySerializer.java | 3 +- .../base/array/StringArraySerializer.java | 3 +- .../typeutils/runtime/AvroSerializer.java | 144 +++- .../runtime/CopyableValueSerializer.java | 42 +- .../typeutils/runtime/EitherSerializer.java | 39 + .../EitherSerializerConfigSnapshot.java | 49 ++ .../typeutils/runtime/KryoRegistration.java | 173 +++++ ...oRegistrationSerializerConfigSnapshot.java | 251 +++++++ .../api/java/typeutils/runtime/KryoUtils.java | 27 + .../typeutils/runtime/PojoSerializer.java | 676 +++++++++++++++--- .../java/typeutils/runtime/RowSerializer.java | 81 ++- .../runtime/TupleSerializerBase.java | 42 ++ .../TupleSerializerConfigSnapshot.java | 91 +++ .../typeutils/runtime/ValueSerializer.java | 86 ++- .../runtime/kryo/KryoSerializer.java | 233 ++++-- .../common/typeutils/SerializerTestBase.java | 53 ++ .../TypeSerializerConfigSnapshotTest.java | 197 +++++ .../typeutils/base/EnumSerializerTest.java | 140 ++++ .../CharPrimitiveArraySerializerTest.java | 2 - .../typeutils/runtime/PojoSerializerTest.java | 264 ++++++- .../kryo/KryoSerializerMigrationTest.java | 136 ++++ .../java/io/CollectionInputFormatTest.java | 12 + .../cep/NonDuplicatingTypeSerializer.java | 16 +- .../java/org/apache/flink/cep/nfa/NFA.java | 47 +- .../AbstractKeyedCEPPatternOperator.java | 35 + .../valuearray/IntValueArraySerializer.java | 7 + .../valuearray/LongValueArraySerializer.java | 7 + .../StringValueArraySerializer.java | 7 + .../table/runtime/types/CRowSerializer.scala | 49 +- .../MigrationNamespaceSerializerProxy.java | 14 + .../runtime/state/ArrayListSerializer.java | 35 +- .../runtime/state/HashMapSerializer.java | 44 +- .../flink/runtime/state/JavaSerializer.java | 19 +- .../testutils/types/IntListSerializer.java | 12 + .../testutils/types/IntPairSerializer.java | 12 + .../testutils/types/StringPairSerializer.java | 12 + .../runtime/state/StateBackendTestBase.java | 2 + .../state/heap/CopyOnWriteStateTableTest.java | 12 + .../recordutils/RecordSerializer.java | 12 + .../scala/typeutils/EitherSerializer.scala | 48 +- .../scala/typeutils/EnumValueSerializer.scala | 116 ++- .../scala/typeutils/NothingSerializer.scala | 12 +- .../scala/typeutils/OptionSerializer.scala | 52 +- .../typeutils/TraversableSerializer.scala | 13 +- .../api/scala/typeutils/TrySerializer.scala | 58 +- .../MultiplexingStreamRecordSerializer.java | 56 ++ .../streamrecord/StreamRecordSerializer.java | 53 ++ .../api/datastream/CoGroupedStreams.java | 12 + .../api/operators/InternalTimer.java | 12 + .../api/windowing/windows/GlobalWindow.java | 18 +- .../api/windowing/windows/TimeWindow.java | 19 +- .../streamrecord/StreamElementSerializer.java | 58 ++ .../CheckpointingCustomKvStateProgram.java | 1 - 99 files changed, 4888 insertions(+), 315 deletions(-) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializerConfigSnapshot.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/ForwardCompatibleSerializationFormatConfig.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/GenericTypeSerializerConfigSnapshot.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/MigrationStrategy.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/ParameterlessTypeSerializerConfig.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshot.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerUtil.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CollectionSerializerConfigSnapshot.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializerConfigSnapshot.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializerConfigSnapshot.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerConfigSnapshot.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoRegistration.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoRegistrationSerializerConfigSnapshot.java create mode 100644 flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerConfigSnapshot.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotTest.java create mode 100644 flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerMigrationTest.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 9036d751c406a..2576cb063fe42 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 @@ -20,7 +20,11 @@ import com.esotericsoftware.kryo.Kryo; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.GenericTypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.util.InstantiationUtil; @@ -30,7 +34,8 @@ import java.io.IOException; -public class WritableSerializer extends TypeSerializer { +@Internal +public final class WritableSerializer extends TypeSerializer { private static final long serialVersionUID = 1L; @@ -149,4 +154,42 @@ public boolean equals(Object obj) { public boolean canEqual(Object obj) { return obj instanceof WritableSerializer; } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public WritableSerializerConfigSnapshot snapshotConfiguration() { + return new WritableSerializerConfigSnapshot<>(typeClass); + } + + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof WritableSerializerConfigSnapshot + && typeClass.equals(((WritableSerializerConfigSnapshot) configSnapshot).getTypeClass())) { + + return MigrationStrategy.noMigration(); + } else { + return MigrationStrategy.migrate(); + } + } + + public static final class WritableSerializerConfigSnapshot + extends GenericTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public WritableSerializerConfigSnapshot() {} + + public WritableSerializerConfigSnapshot(Class writableTypeClass) { + super(writableTypeClass); + } + + @Override + public int getVersion() { + return VERSION; + } + } } 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 new file mode 100644 index 0000000000000..e7e2650c4139c --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompositeTypeSerializerConfigSnapshot.java @@ -0,0 +1,85 @@ +/* + * 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; +import java.util.Arrays; + +/** + * A {@link TypeSerializerConfigSnapshot} for serializers that has multiple nested serializers. + * The configuration snapshot consists of the configuration snapshots of all nested serializers. + */ +@Internal +public abstract class CompositeTypeSerializerConfigSnapshot extends TypeSerializerConfigSnapshot { + + private TypeSerializerConfigSnapshot[] nestedSerializerConfigSnapshots; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public CompositeTypeSerializerConfigSnapshot() {} + + public CompositeTypeSerializerConfigSnapshot(TypeSerializerConfigSnapshot... nestedSerializerConfigSnapshots) { + this.nestedSerializerConfigSnapshots = Preconditions.checkNotNull(nestedSerializerConfigSnapshots); + } + + @Override + public void write(DataOutputView out) throws IOException { + super.write(out); + TypeSerializerUtil.writeSerializerConfigSnapshots(out, nestedSerializerConfigSnapshots); + } + + @Override + public void read(DataInputView in) throws IOException { + super.read(in); + nestedSerializerConfigSnapshots = TypeSerializerUtil.readSerializerConfigSnapshots(in, getUserCodeClassLoader()); + } + + public TypeSerializerConfigSnapshot[] getNestedSerializerConfigSnapshots() { + return nestedSerializerConfigSnapshots; + } + + public TypeSerializerConfigSnapshot getSingleNestedSerializerConfigSnapshot() { + return nestedSerializerConfigSnapshots[0]; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + + if (obj == null) { + return false; + } + + return (obj.getClass().equals(getClass())) + && Arrays.equals( + nestedSerializerConfigSnapshots, + ((CompositeTypeSerializerConfigSnapshot) obj).getNestedSerializerConfigSnapshots()); + } + + @Override + public int hashCode() { + return Arrays.hashCode(nestedSerializerConfigSnapshots); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/ForwardCompatibleSerializationFormatConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/ForwardCompatibleSerializationFormatConfig.java new file mode 100644 index 0000000000000..4d81e31c503aa --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/ForwardCompatibleSerializationFormatConfig.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.typeutils; + +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; + +/** + * This is a special {@link TypeSerializerConfigSnapshot} that serializers can return to serve + * as a marker to indicate that new serializers for the data written by this serializer does not + * need to be checked for compatibility. + */ +@PublicEvolving +public final class ForwardCompatibleSerializationFormatConfig extends TypeSerializerConfigSnapshot { + + /** Singleton instance. */ + public static final ForwardCompatibleSerializationFormatConfig INSTANCE = + new ForwardCompatibleSerializationFormatConfig(); + + @Override + public void write(DataOutputView out) throws IOException { + // nothing to write + } + + @Override + public void read(DataInputView in) throws IOException { + // nothing to read + } + + @Override + public int getSnapshotVersion() { + throw new UnsupportedOperationException( + "This is a ForwardCompatibleSerializationFormatConfig. No versioning required."); + } + + @Override + public int getVersion() { + throw new UnsupportedOperationException( + "This is a ForwardCompatibleSerializationFormatConfig. No versioning required."); + } + + /** + * This special configuration type does not require the default + * empty nullary constructor because it will never actually be serialized. + */ + private ForwardCompatibleSerializationFormatConfig() {} + + @Override + public boolean equals(Object obj) { + return obj instanceof ForwardCompatibleSerializationFormatConfig; + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } +} 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 new file mode 100644 index 0000000000000..4edfe12324134 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/GenericTypeSerializerConfigSnapshot.java @@ -0,0 +1,88 @@ +/* + * 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; + +/** + * Configuration snapshot for serializers for generic types. + * + * @param The type to be instantiated. + */ +@Internal +public abstract class GenericTypeSerializerConfigSnapshot extends TypeSerializerConfigSnapshot { + + private Class typeClass; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public GenericTypeSerializerConfigSnapshot() {} + + public GenericTypeSerializerConfigSnapshot(Class typeClass) { + this.typeClass = Preconditions.checkNotNull(typeClass); + } + + @Override + public void write(DataOutputView out) throws IOException { + super.write(out); + + // write only the classname to avoid Java serialization + out.writeUTF(typeClass.getName()); + } + + @SuppressWarnings("unchecked") + @Override + public void read(DataInputView in) throws IOException { + super.read(in); + + String genericTypeClassname = in.readUTF(); + try { + typeClass = (Class) Class.forName(genericTypeClassname, true, getUserCodeClassLoader()); + } catch (ClassNotFoundException e) { + throw new IOException("Could not find the requested class " + genericTypeClassname + " in classpath.", e); + } + } + + public Class getTypeClass() { + return typeClass; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + + if (obj == null) { + return false; + } + + return (obj.getClass().equals(getClass())) + && typeClass.equals(((GenericTypeSerializerConfigSnapshot) obj).getTypeClass()); + } + + @Override + public int hashCode() { + return typeClass.hashCode(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/MigrationStrategy.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/MigrationStrategy.java new file mode 100644 index 0000000000000..c76027b897c5e --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/MigrationStrategy.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.typeutils; + +import org.apache.flink.annotation.PublicEvolving; + +/** + * A {@code MigrationStrategy} contains information about how to perform migration of data written + * by an older serializer so that new serializers can continue to work on them. + * + * @param the type of the data being migrated. + */ +@PublicEvolving +public final class MigrationStrategy { + + /** Whether or not migration is required. */ + private final boolean requiresStateMigration; + + /** + * The fallback deserializer to use, in the case the preceding serializer cannot be found. + * + *

This is only relevant if migration is required. + */ + private final TypeSerializer fallbackDeserializer; + + /** + * Returns a strategy that simply signals that no migration needs to be performed. + * + * @return a strategy that does not perform migration + */ + public static MigrationStrategy noMigration() { + return new MigrationStrategy<>(false, null); + } + + /** + * Returns a strategy that signals migration to be performed, and in the case that the + * preceding serializer cannot be found, a provided fallback deserializer can be + * used. + * + * @param fallbackDeserializer a fallback deserializer that can be used to read old data for the migration + * in the case that the preceding serializer cannot be found. + * + * @return a strategy that performs migration with a fallback deserializer to read old data. + */ + public static MigrationStrategy migrateWithFallbackDeserializer(TypeSerializer fallbackDeserializer) { + return new MigrationStrategy<>(true, fallbackDeserializer); + } + + /** + * Returns a strategy that signals migration to be performed, without a fallback deserializer. + * If the preceding serializer cannot be found, the migration fails because the old data cannot be read. + * + * @return a strategy that performs migration, without a fallback deserializer. + */ + public static MigrationStrategy migrate() { + return new MigrationStrategy<>(true, null); + } + + private MigrationStrategy(boolean requiresStateMigration, TypeSerializer fallbackDeserializer) { + this.requiresStateMigration = requiresStateMigration; + this.fallbackDeserializer = fallbackDeserializer; + } + + public TypeSerializer getFallbackDeserializer() { + return fallbackDeserializer; + } + + public boolean requireMigration() { + return requiresStateMigration; + } +} 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 new file mode 100644 index 0000000000000..7ba7dd452d609 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/ParameterlessTypeSerializerConfig.java @@ -0,0 +1,89 @@ +/* + * 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 base class for {@link TypeSerializerConfigSnapshot}s that do not have any parameters. + */ +@Internal +public final class ParameterlessTypeSerializerConfig extends TypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** + * A string identifier that encodes the serialization format used by the serializer. + * + * TODO we might change this to a proper serialization format class in the future + */ + private String serializationFormatIdentifier; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public ParameterlessTypeSerializerConfig() {} + + public ParameterlessTypeSerializerConfig(String serializationFormatIdentifier) { + this.serializationFormatIdentifier = Preconditions.checkNotNull(serializationFormatIdentifier); + } + + @Override + public void write(DataOutputView out) throws IOException { + super.write(out); + out.writeUTF(serializationFormatIdentifier); + } + + @Override + public void read(DataInputView in) throws IOException { + super.read(in); + serializationFormatIdentifier = in.readUTF(); + } + + @Override + public int getVersion() { + return VERSION; + } + + public String getSerializationFormatIdentifier() { + return serializationFormatIdentifier; + } + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } + + if (other == null) { + return false; + } + + return (other instanceof ParameterlessTypeSerializerConfig) + && serializationFormatIdentifier.equals(((ParameterlessTypeSerializerConfig) other).getSerializationFormatIdentifier()); + } + + @Override + public int hashCode() { + return serializationFormatIdentifier.hashCode(); + } +} 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 6edaec6a44e2a..938da3b519410 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; @@ -164,4 +165,94 @@ public abstract class TypeSerializer implements Serializable { public boolean canRestoreFrom(TypeSerializer other) { return equals(other); } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + /** + * Create a snapshot of the serializer's current configuration to be stored along with the managed state it is + * registered to (if any - this method is only relevant if this serializer is registered for serialization of + * managed state). + * + *

The configuration snapshot should contain information about the serializer's parameter settings and its + * serialization format. When a new serializer is registered to serialize the same managed state that this + * serializer was registered to, the returned configuration snapshot can be used to check with the new serializer + * if any data migration needs to take place. + * + *

Implementations can also return the singleton {@link ForwardCompatibleSerializationFormatConfig#INSTANCE} + * configuration if they guarantee forwards compatibility. For example, implementations that use serialization + * frameworks with built-in serialization compatibility, such as Thrift or + * Protobuf, is suitable for this usage pattern. By + * returning the {@link ForwardCompatibleSerializationFormatConfig#INSTANCE}, this informs Flink that when managed + * state serialized using this serializer is restored, there is no need to check for migration with the new + * serializer for the same state. In other words, new serializers are always assumed to be fully compatible for the + * serialized state. + * + * @see TypeSerializerConfigSnapshot + * @see ForwardCompatibleSerializationFormatConfig + * + * @return snapshot of the serializer's current configuration. + */ + public abstract TypeSerializerConfigSnapshot snapshotConfiguration(); + + /** + * Get the migration strategy to use this serializer based on the configuration snapshot of 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). + * + *

Implementations need to return the resolved migration strategy. The strategy can be one of the following: + *

    + *
  • {@link MigrationStrategy#noMigration()}: this signals Flink that this serializer is compatible, or + * has been reconfigured to be compatible, to continue reading old data, and that the + * serialization schema remains the same. No migration needs to be performed.
  • + * + *
  • {@link MigrationStrategy#migrateWithFallbackDeserializer(TypeSerializer)}: this signals Flink that + * migration needs to be performed, because this serializer is not compatible, or cannot be reconfigured to be + * compatible, for old data. Furthermore, in the case that the preceding serializer cannot be found or + * restored to read the old data, the provided fallback deserializer can be used.
  • + * + *
  • {@link MigrationStrategy#migrate()}: this signals Flink that migration needs to be performed, because + * this serializer is not compatible, or cannot be reconfigured to be compatible, for old data.
  • + *
+ * + *

This method is guaranteed to only be invoked if the preceding serializer's configuration snapshot is not the + * singleton {@link ForwardCompatibleSerializationFormatConfig#INSTANCE} configuration. In such cases, Flink always + * assume that the migration strategy is {@link MigrationStrategy#migrate()}. + * + * @see MigrationStrategy + * + * @param configSnapshot configuration snapshot of a preceding serializer for the same managed state + * + * @return the result of the reconfiguration. + */ + protected abstract MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot); + + /** + * Get the migration strategy to use this serializer based on the configuration snapshot of 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). + * + *

This method is not part of the public user-facing API, and cannot be overriden. External operations + * providing a configuration snapshot of preceding serializer can only do so through this method. + * + *

This method always assumes that the migration strategy is {@link MigrationStrategy#noMigration()} if + * the provided configuration snapshot is the singleton {@link ForwardCompatibleSerializationFormatConfig#INSTANCE}. + * Otherwise, the configuration snapshot is provided to the actual + * {@link #getMigrationStrategy(TypeSerializerConfigSnapshot)} (TypeSerializerConfigSnapshot)} implementation. + * + * @param configSnapshot configuration snapshot of a preceding serializer for the same managed state + * + * @return the result of the reconfiguration. + */ + @Internal + public final MigrationStrategy getMigrationStrategyFor(TypeSerializerConfigSnapshot configSnapshot) { + // reference equality is viable here, because the forward compatible + // marker config will always be explicitly restored with the singleton instance + if (configSnapshot != ForwardCompatibleSerializationFormatConfig.INSTANCE) { + return getMigrationStrategy(configSnapshot); + } else { + return MigrationStrategy.noMigration(); + } + } } 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 new file mode 100644 index 0000000000000..1148b0801fd71 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshot.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.typeutils; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.core.io.VersionMismatchException; +import org.apache.flink.core.io.VersionedIOReadableWritable; +import org.apache.flink.util.Preconditions; + +/** + * 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 persisted configuration may later on be used by new serializers to decide the migration strategy for the + * same managed state. In order for new serializers to be able to decide migration strategy, the configuration snapshot + * should encode sufficient information about: + * + *

    + *
  • Parameter settings of the serializer: parameters of the serializer include settings + * required to setup the serializer, or the state of the serializer if it is stateful. If the serializer + * 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.
  • + *
+ * + *

NOTE: Implementations must contain the default empty nullary constructor. This is required to be able to + * deserialize the configuration snapshot from its binary form. + */ +@PublicEvolving +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; + + /** The snapshot version of this configuration. */ + private Integer snapshotVersion; + + /** + * Returns the version of the configuration at the time its snapshot was taken. + * + * @return the snapshot configuration's version. + */ + public int getSnapshotVersion() { + if (snapshotVersion == null) { + return getVersion(); + } else { + return snapshotVersion; + } + } + + /** + * Set the user code class loader. + * Only relevant if this configuration instance was deserialized from binary form. + * + *

This method is not part of the public user-facing API, and cannot be overriden. + * + * @param userCodeClassLoader user code class loader. + */ + @Internal + public final void setUserCodeClassLoader(ClassLoader userCodeClassLoader) { + this.userCodeClassLoader = Preconditions.checkNotNull(userCodeClassLoader); + } + + /** + * Returns the user code class loader. + * Only relevant if this configuration instance was deserialized from binary form. + * + * @return the user code class loader + */ + @Internal + public final ClassLoader getUserCodeClassLoader() { + return userCodeClassLoader; + } + + @Override + protected void resolveVersionRead(int foundVersion) throws VersionMismatchException { + super.resolveVersionRead(foundVersion); + this.snapshotVersion = foundVersion; + } + + public abstract boolean equals(Object obj); + + public abstract int hashCode(); +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java index cb8967b207a60..f36021897b19a 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java @@ -205,6 +205,16 @@ public void copy(DataInputView source, DataOutputView target) throws IOException throw new UnsupportedOperationException("This object is a dummy TypeSerializer."); } + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + throw new UnsupportedOperationException("This object is a dummy TypeSerializer."); + } + + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + throw new UnsupportedOperationException("This object is a dummy TypeSerializer."); + } + @Override public boolean canEqual(Object obj) { return false; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerUtil.java new file mode 100644 index 0000000000000..768d8d2b30070 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerUtil.java @@ -0,0 +1,216 @@ +/* + * 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.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; + +/** + * Utility methods for {@link TypeSerializer} and {@link TypeSerializerConfigSnapshot}. + */ +@Internal +public class TypeSerializerUtil { + + /** + * Creates an array of {@link TypeSerializerConfigSnapshot}s taken + * from the provided array of {@link TypeSerializer}s. + * + * @param serializers array of type serializers. + * + * @return array of configuration snapshots taken from each serializer. + */ + public static TypeSerializerConfigSnapshot[] snapshotConfigurations(TypeSerializer[] serializers) { + final TypeSerializerConfigSnapshot[] configSnapshots = new TypeSerializerConfigSnapshot[serializers.length]; + + for (int i = 0; i < serializers.length; i++) { + configSnapshots[i] = serializers[i].snapshotConfiguration(); + } + + return configSnapshots; + } + + /** + * 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; {@code null} is handled + * + * @throws IOException + */ + public static void writeSerializerConfigSnapshot( + DataOutputView out, + TypeSerializerConfigSnapshot serializerConfigSnapshot) throws IOException { + + new TypeSerializerConfigSnapshotProxy(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 TypeSerializerConfigSnapshotProxy proxy = new TypeSerializerConfigSnapshotProxy(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; {@code null} is handled + * + * @throws IOException + */ + public static void writeSerializerConfigSnapshots( + DataOutputView out, + TypeSerializerConfigSnapshot... serializerConfigSnapshots) throws IOException { + + out.writeInt(serializerConfigSnapshots.length); + + for (TypeSerializerConfigSnapshot snapshot : serializerConfigSnapshots) { + new TypeSerializerConfigSnapshotProxy(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]; + + TypeSerializerConfigSnapshotProxy proxy; + for (int i = 0; i < numFields; i++) { + proxy = new TypeSerializerConfigSnapshotProxy(userCodeClassLoader); + proxy.read(in); + serializerConfigSnapshots[i] = proxy.getSerializerConfigSnapshot(); + } + + return serializerConfigSnapshots; + } + + /** + * Utility serialization proxy for a {@link TypeSerializerConfigSnapshot}. + */ + static class TypeSerializerConfigSnapshotProxy extends VersionedIOReadableWritable { + + private static final int VERSION = 1; + + private ClassLoader userCodeClassLoader; + private TypeSerializerConfigSnapshot serializerConfigSnapshot; + + TypeSerializerConfigSnapshotProxy(ClassLoader userCodeClassLoader) { + this.userCodeClassLoader = Preconditions.checkNotNull(userCodeClassLoader); + } + + TypeSerializerConfigSnapshotProxy(TypeSerializerConfigSnapshot serializerConfigSnapshot) { + this.serializerConfigSnapshot = serializerConfigSnapshot; + } + + @Override + public void write(DataOutputView out) throws IOException { + super.write(out); + + if (serializerConfigSnapshot == ForwardCompatibleSerializationFormatConfig.INSTANCE) { + // if the config is actually the special forward compatible config, + // just write a flag to indicate that and write nothing else + out.writeBoolean(false); + } else { + out.writeBoolean(true); + + // 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); + + if (in.readBoolean()) { + 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); + } else { + // was a special forward compatible config; restore with the singleton instance + serializerConfigSnapshot = ForwardCompatibleSerializationFormatConfig.INSTANCE; + } + } + + @Override + public int getVersion() { + return VERSION; + } + + TypeSerializerConfigSnapshot getSerializerConfigSnapshot() { + return serializerConfigSnapshot; + } + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanSerializer.java index 609f1849d595c..f275807b8b882 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanSerializer.java @@ -82,4 +82,10 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof BooleanSerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(BooleanValueSerializer.class.getCanonicalName()); + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanValueSerializer.java index 62c91dfb86922..4755549bbc3b4 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanValueSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/BooleanValueSerializer.java @@ -31,8 +31,7 @@ public final class BooleanValueSerializer extends TypeSerializerSingleton { private static final Byte ZERO = Byte.valueOf((byte) 0); - @Override public boolean isImmutableType() { return true; @@ -83,4 +82,10 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof ByteSerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(ByteValueSerializer.class.getCanonicalName()); + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteValueSerializer.java index 848b01e0860f5..2547dda6f7ebf 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteValueSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ByteValueSerializer.java @@ -31,8 +31,7 @@ public final class ByteValueSerializer extends TypeSerializerSingleton { private static final Character ZERO = Character.valueOf((char)0); - @Override public boolean isImmutableType() { return true; @@ -83,4 +82,10 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof CharSerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(CharValueSerializer.class.getCanonicalName()); + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharValueSerializer.java index 84dc39a747999..e012b8c2f712b 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharValueSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CharValueSerializer.java @@ -31,8 +31,7 @@ public class CharValueSerializer extends TypeSerializerSingleton { private static final long serialVersionUID = 1L; public static final CharValueSerializer INSTANCE = new CharValueSerializer(); - - + @Override public boolean isImmutableType() { return false; @@ -84,4 +83,10 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof CharValueSerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(CharSerializer.class.getCanonicalName()); + } } 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 new file mode 100644 index 0000000000000..8fa2315dddd04 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/CollectionSerializerConfigSnapshot.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.api.common.typeutils.base; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; + +/** + * Configuration snapshot of a serializer for collection types. + */ +@Internal +public final class CollectionSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public CollectionSerializerConfigSnapshot() {} + + public CollectionSerializerConfigSnapshot(TypeSerializerConfigSnapshot elementSerializerConfigSnapshot) { + super(elementSerializerConfigSnapshot); + } + + @Override + public int getVersion() { + return VERSION; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DateSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DateSerializer.java index 3f27de2f8b1ff..28ed904208c29 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DateSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DateSerializer.java @@ -101,4 +101,11 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof DateSerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(SqlDateSerializer.class.getCanonicalName()) + || identifier.equals(SqlTimeSerializer.class.getCanonicalName()); + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleSerializer.java index 375cb9c0738a0..92fe71d55a797 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleSerializer.java @@ -33,7 +33,6 @@ public final class DoubleSerializer extends TypeSerializerSingleton { private static final Double ZERO = Double.valueOf(0); - @Override public boolean isImmutableType() { return true; @@ -83,4 +82,10 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof DoubleSerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(DoubleValueSerializer.class.getCanonicalName()); + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleValueSerializer.java index 232ad6b962596..9e7e8d0e164fa 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleValueSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/DoubleValueSerializer.java @@ -31,8 +31,7 @@ public final class DoubleValueSerializer extends TypeSerializerSingleton> extends TypeSerializer { private final Class enumClass; - private transient T[] values; + /** + * Maintain our own map of enum value to their ordinal, instead of directly using {@link Enum#ordinal}. + * This allows us to maintain backwards compatibility for previous serialized data in the case that the + * order of enum constants was changed or new constants were added. + * + *

On a fresh start with no reconfiguration, the ordinals would simply be identical to the enum + * constants actual ordinals. Ordinals may change after reconfiguration. + */ + private Map valueToOrdinal; + + /** + * Array of enum constants with their indexes identical to their ordinals in the {@link #valueToOrdinal} map. + * Serves as a bidirectional map to have fast access from ordinal to value. May be reordered after reconfiguration. + */ + private T[] values; public EnumSerializer(Class enumClass) { this.enumClass = checkNotNull(enumClass); @@ -44,6 +71,12 @@ public EnumSerializer(Class enumClass) { this.values = enumClass.getEnumConstants(); checkArgument(this.values.length > 0, "cannot use an empty enum"); + + this.valueToOrdinal = new HashMap<>(values.length); + int i = 0; + for (T value : values) { + this.valueToOrdinal.put(value, i++); + } } @Override @@ -78,7 +111,8 @@ public int getLength() { @Override public void serialize(T record, DataOutputView target) throws IOException { - target.writeInt(record.ordinal()); + // use our own maintained ordinals instead of the actual enum ordinal + target.writeInt(valueToOrdinal.get(record)); } @Override @@ -121,6 +155,141 @@ public int hashCode() { private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { in.defaultReadObject(); - this.values = enumClass.getEnumConstants(); + + // may be null if this serializer was deserialized from an older version + if (this.values == null) { + this.values = enumClass.getEnumConstants(); + + this.valueToOrdinal = new HashMap<>(values.length); + int i = 0; + for (T value : values) { + this.valueToOrdinal.put(value, i++); + } + } + } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public EnumSerializerConfigSnapshot snapshotConfiguration() { + return new EnumSerializerConfigSnapshot<>(enumClass, values); + } + + @SuppressWarnings("unchecked") + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof EnumSerializerConfigSnapshot) { + final EnumSerializerConfigSnapshot config = (EnumSerializerConfigSnapshot) configSnapshot; + + if (enumClass.equals(config.getTypeClass())) { + + // reorder enum constants so that previously existing constants + // remain in the same order, and new + LinkedHashSet reorderedEnumConstants = new LinkedHashSet<>(); + reorderedEnumConstants.addAll(Arrays.asList(config.getEnumConstants())); + reorderedEnumConstants.addAll(Arrays.asList(enumClass.getEnumConstants())); + + // regenerate enum constant to ordinal bidirectional map + this.values = (T[]) Array.newInstance(enumClass, reorderedEnumConstants.size()); + this.valueToOrdinal.clear(); + int i = 0; + for (T constant : reorderedEnumConstants) { + this.values[i] = constant; + this.valueToOrdinal.put(constant, i); + i++; + } + + return MigrationStrategy.noMigration(); + } + } + + return MigrationStrategy.migrate(); + } + + /** + * Configuration snapshot of a serializer for enumerations. + * + * Configuration contains the enum class, and an array of the enum's constants + * that existed when the configuration snapshot was taken. + * + * @param the enum type. + */ + public static final class EnumSerializerConfigSnapshot> + extends GenericTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + private T[] enumConstants; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public EnumSerializerConfigSnapshot() {} + + public EnumSerializerConfigSnapshot(Class enumClass, T[] enumConstants) { + super(enumClass); + this.enumConstants = Preconditions.checkNotNull(enumConstants); + } + + @Override + public void write(DataOutputView out) throws IOException { + super.write(out); + + try (final DataOutputViewStream outViewWrapper = new DataOutputViewStream(out)) { + InstantiationUtil.serializeObject(outViewWrapper, enumConstants); + } + } + + @Override + public void read(DataInputView in) throws IOException { + super.read(in); + + try (final DataInputViewStream inViewWrapper = new DataInputViewStream(in)) { + try { + enumConstants = InstantiationUtil.deserializeObject(inViewWrapper, getUserCodeClassLoader()); + } catch (ClassNotFoundException e) { + throw new IOException("The requested enum class cannot be found in classpath.", e); + } catch (IllegalArgumentException e) { + throw new IOException("A previously existing enum constant of " + + getTypeClass().getName() + " no longer exists.", e); + } + } + } + + @Override + public int getVersion() { + return VERSION; + } + + public T[] getEnumConstants() { + return enumConstants; + } + + @Override + public boolean equals(Object obj) { + return super.equals(obj) + && Arrays.equals( + enumConstants, + ((EnumSerializerConfigSnapshot) obj).getEnumConstants()); + } + + @Override + public int hashCode() { + return super.hashCode() * 31 + Arrays.hashCode(enumConstants); + } + } + + // -------------------------------------------------------------------------------------------- + // Test utilities + // -------------------------------------------------------------------------------------------- + + @VisibleForTesting + T[] getValues() { + return values; + } + + @VisibleForTesting + Map getValueToOrdinal() { + return valueToOrdinal; } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatSerializer.java index b1550054832df..953c4fa979a5c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatSerializer.java @@ -33,7 +33,6 @@ public final class FloatSerializer extends TypeSerializerSingleton { private static final Float ZERO = Float.valueOf(0); - @Override public boolean isImmutableType() { return true; @@ -83,4 +82,10 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof FloatSerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(FloatValueSerializer.class.getCanonicalName()); + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatValueSerializer.java index 3ccd88a021734..2542be9e4f0e9 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatValueSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/FloatValueSerializer.java @@ -31,8 +31,7 @@ public class FloatValueSerializer extends TypeSerializerSingleton { private static final long serialVersionUID = 1L; public static final FloatValueSerializer INSTANCE = new FloatValueSerializer(); - - + @Override public boolean isImmutableType() { return false; @@ -84,4 +83,10 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof FloatValueSerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(FloatSerializer.class.getCanonicalName()); + } } 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 f35d71b7d7fa2..b8ffd4a158246 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializer.java @@ -22,7 +22,9 @@ import java.lang.reflect.Array; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -31,7 +33,7 @@ /** * A serializer for arrays of objects. * - * @param The component type + * @param The component type. */ @Internal public final class GenericArraySerializer extends TypeSerializer { @@ -186,4 +188,40 @@ public boolean canEqual(Object obj) { public String toString() { return "Serializer " + componentClass.getName() + "[]"; } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public GenericArraySerializerConfigSnapshot snapshotConfiguration() { + return new GenericArraySerializerConfigSnapshot<>(componentClass, componentSerializer.snapshotConfiguration()); + } + + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof GenericArraySerializerConfigSnapshot) { + final GenericArraySerializerConfigSnapshot config = (GenericArraySerializerConfigSnapshot) configSnapshot; + + if (componentClass.equals(config.getComponentClass())) { + MigrationStrategy strategy = componentSerializer.getMigrationStrategyFor( + config.getSingleNestedSerializerConfigSnapshot()); + + if (strategy.requireMigration()) { + if (strategy.getFallbackDeserializer() != null) { + return MigrationStrategy.migrateWithFallbackDeserializer( + new GenericArraySerializer<>( + componentClass, + strategy.getFallbackDeserializer())); + } else { + return MigrationStrategy.migrate(); + } + } else { + return MigrationStrategy.noMigration(); + } + } + } + + return MigrationStrategy.migrate(); + } } 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 new file mode 100644 index 0000000000000..e78eb6c01ee93 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializerConfigSnapshot.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.typeutils.base; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream; +import org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; + +/** + * Point-in-time configuration of a {@link GenericArraySerializer}. + * + * @param The component type. + */ +@Internal +public final class GenericArraySerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + private Class componentClass; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public GenericArraySerializerConfigSnapshot() {} + + public GenericArraySerializerConfigSnapshot( + Class componentClass, + TypeSerializerConfigSnapshot componentSerializerConfigSnapshot) { + + super(componentSerializerConfigSnapshot); + + this.componentClass = Preconditions.checkNotNull(componentClass); + } + + @Override + public void write(DataOutputView out) throws IOException { + super.write(out); + + InstantiationUtil.serializeObject(new DataOutputViewStream(out), componentClass); + } + + @Override + public void read(DataInputView in) throws IOException { + super.read(in); + + try { + componentClass = InstantiationUtil.deserializeObject(new DataInputViewStream(in), getUserCodeClassLoader()); + } catch (ClassNotFoundException e) { + throw new IOException("Could not find requested element class in classpath.", e); + } + } + + @Override + public int getVersion() { + return VERSION; + } + + public Class getComponentClass() { + return componentClass; + } + + @Override + public boolean equals(Object obj) { + return super.equals(obj) + && (obj instanceof GenericArraySerializerConfigSnapshot) + && (componentClass.equals(((GenericArraySerializerConfigSnapshot) obj).getComponentClass())); + } + + @Override + public int hashCode() { + return super.hashCode() * 31 + componentClass.hashCode(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntSerializer.java index de6b3261c9880..e046e8eb173ef 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntSerializer.java @@ -33,7 +33,6 @@ public final class IntSerializer extends TypeSerializerSingleton { private static final Integer ZERO = Integer.valueOf(0); - @Override public boolean isImmutableType() { return true; @@ -83,4 +82,10 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof IntSerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(IntValueSerializer.class.getCanonicalName()); + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntValueSerializer.java index 1a90313c30307..65f5e359bf283 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntValueSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/IntValueSerializer.java @@ -31,8 +31,7 @@ public final class IntValueSerializer extends TypeSerializerSingleton private static final long serialVersionUID = 1L; public static final IntValueSerializer INSTANCE = new IntValueSerializer(); - - + @Override public boolean isImmutableType() { return false; @@ -84,4 +83,10 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof IntValueSerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(IntSerializer.class.getCanonicalName()); + } } 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 5d5e8f83cf7d8..082e11562b360 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java @@ -19,7 +19,9 @@ package org.apache.flink.api.common.typeutils.base; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -39,7 +41,7 @@ * @param The type of element in the list. */ @Internal -public class ListSerializer extends TypeSerializer> { +public final class ListSerializer extends TypeSerializer> { private static final long serialVersionUID = 1119562170939152304L; @@ -166,4 +168,34 @@ public boolean canEqual(Object obj) { public int hashCode() { return elementSerializer.hashCode(); } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public CollectionSerializerConfigSnapshot snapshotConfiguration() { + return new CollectionSerializerConfigSnapshot(elementSerializer.snapshotConfiguration()); + } + + @Override + protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof CollectionSerializerConfigSnapshot) { + MigrationStrategy strategy = elementSerializer.getMigrationStrategyFor( + ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); + + if (strategy.requireMigration()) { + if (strategy.getFallbackDeserializer() != null) { + return MigrationStrategy.migrateWithFallbackDeserializer( + new ListSerializer<>(strategy.getFallbackDeserializer())); + } else { + return MigrationStrategy.migrate(); + } + } else { + return MigrationStrategy.noMigration(); + } + } else { + return MigrationStrategy.migrate(); + } + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongSerializer.java index 086a2fc2c7cec..cbdc3db747638 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongSerializer.java @@ -33,7 +33,6 @@ public final class LongSerializer extends TypeSerializerSingleton { private static final Long ZERO = Long.valueOf(0); - @Override public boolean isImmutableType() { return true; @@ -83,4 +82,10 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof LongSerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(LongValueSerializer.class.getCanonicalName()); + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongValueSerializer.java index 5d943251edf69..dcf3805c07a02 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongValueSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/LongValueSerializer.java @@ -31,8 +31,7 @@ public final class LongValueSerializer extends TypeSerializerSingleton The type of the keys in the map. * @param The type of the values in the map. */ -public class MapSerializer extends TypeSerializer> { +@Internal +public final class MapSerializer extends TypeSerializer> { private static final long serialVersionUID = -6885593032367050078L; @@ -190,4 +194,41 @@ public boolean canEqual(Object obj) { public int hashCode() { return keySerializer.hashCode() * 31 + valueSerializer.hashCode(); } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public MapSerializerConfigSnapshot snapshotConfiguration() { + return new MapSerializerConfigSnapshot( + keySerializer.snapshotConfiguration(), + valueSerializer.snapshotConfiguration()); + } + + @Override + protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof MapSerializerConfigSnapshot) { + TypeSerializerConfigSnapshot[] keyValueSerializerConfigSnapshots = + ((MapSerializerConfigSnapshot) configSnapshot).getNestedSerializerConfigSnapshots(); + + MigrationStrategy keyStrategy = keySerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[0]); + MigrationStrategy valueStrategy = valueSerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[1]); + + if (keyStrategy.requireMigration() || valueStrategy.requireMigration()) { + if (keyStrategy.getFallbackDeserializer() != null && valueStrategy.getFallbackDeserializer() != null) { + return MigrationStrategy.migrateWithFallbackDeserializer( + new MapSerializer<>( + keyStrategy.getFallbackDeserializer(), + valueStrategy.getFallbackDeserializer())); + } else { + return MigrationStrategy.migrate(); + } + } else { + return MigrationStrategy.noMigration(); + } + } else { + return MigrationStrategy.migrate(); + } + } } 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 new file mode 100644 index 0000000000000..38e12545d8def --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializerConfigSnapshot.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.api.common.typeutils.base; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; + +/** + * Configuration snapshot for serializers of maps, containing the + * configuration snapshot of its key serializer and value serializer. + */ +@Internal +public final class MapSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public MapSerializerConfigSnapshot() {} + + public MapSerializerConfigSnapshot( + TypeSerializerConfigSnapshot keySerializerConfigSnapshot, + TypeSerializerConfigSnapshot valueSerializerConfigSnapshot) { + + super(keySerializerConfigSnapshot, valueSerializerConfigSnapshot); + } + + @Override + public int getVersion() { + return VERSION; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortSerializer.java index b94c0ac1e4aa9..322591d8563d9 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortSerializer.java @@ -33,7 +33,6 @@ public final class ShortSerializer extends TypeSerializerSingleton { private static final Short ZERO = Short.valueOf((short)0); - @Override public boolean isImmutableType() { return true; @@ -83,4 +82,10 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof ShortSerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(ShortValueSerializer.class.getCanonicalName()); + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortValueSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortValueSerializer.java index 51b234ef36b99..9e909279db366 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortValueSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ShortValueSerializer.java @@ -31,8 +31,7 @@ public final class ShortValueSerializer extends TypeSerializerSingleton extends TypeSerializer{ @@ -48,4 +51,33 @@ public boolean equals(Object obj) { return false; } } + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + // type serializer singletons should always be parameter-less + return new ParameterlessTypeSerializerConfig(getSerializationFormatIdentifier()); + } + + @Override + public MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof ParameterlessTypeSerializerConfig + && isCompatibleSerializationFormatIdentifier( + ((ParameterlessTypeSerializerConfig) configSnapshot).getSerializationFormatIdentifier())) { + + return MigrationStrategy.noMigration(); + } else { + return MigrationStrategy.migrate(); + } + } + + /** + * Subclasses can override this if they know that they are also compatible with identifiers of other formats. + */ + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return identifier.equals(getSerializationFormatIdentifier()); + } + + private String getSerializationFormatIdentifier() { + return getClass().getCanonicalName(); + } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BooleanPrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BooleanPrimitiveArraySerializer.java index 371200c690bd2..bf14f726da2a0 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BooleanPrimitiveArraySerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BooleanPrimitiveArraySerializer.java @@ -36,8 +36,7 @@ public final class BooleanPrimitiveArraySerializer extends TypeSerializerSinglet private static final boolean[] EMPTY = new boolean[0]; public static final BooleanPrimitiveArraySerializer INSTANCE = new BooleanPrimitiveArraySerializer(); - - + @Override public boolean isImmutableType() { return false; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BytePrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BytePrimitiveArraySerializer.java index 0f56149f9d4b4..dbad1dfc821d2 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BytePrimitiveArraySerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/BytePrimitiveArraySerializer.java @@ -36,7 +36,7 @@ public final class BytePrimitiveArraySerializer extends TypeSerializerSingleton< private static final byte[] EMPTY = new byte[0]; public static final BytePrimitiveArraySerializer INSTANCE = new BytePrimitiveArraySerializer(); - + @Override public boolean isImmutableType() { return false; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/CharPrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/CharPrimitiveArraySerializer.java index c3655e999d121..ae3010911694d 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/CharPrimitiveArraySerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/CharPrimitiveArraySerializer.java @@ -36,8 +36,7 @@ public final class CharPrimitiveArraySerializer extends TypeSerializerSingleton< private static final char[] EMPTY = new char[0]; public static final CharPrimitiveArraySerializer INSTANCE = new CharPrimitiveArraySerializer(); - - + @Override public boolean isImmutableType() { return false; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/DoublePrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/DoublePrimitiveArraySerializer.java index 2b07cfd3fa0fc..610a22b2023b7 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/DoublePrimitiveArraySerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/DoublePrimitiveArraySerializer.java @@ -36,8 +36,7 @@ public final class DoublePrimitiveArraySerializer extends TypeSerializerSingleto private static final double[] EMPTY = new double[0]; public static final DoublePrimitiveArraySerializer INSTANCE = new DoublePrimitiveArraySerializer(); - - + @Override public boolean isImmutableType() { return false; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/FloatPrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/FloatPrimitiveArraySerializer.java index 1fa4715df69c2..50b2b2e9a8d71 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/FloatPrimitiveArraySerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/FloatPrimitiveArraySerializer.java @@ -36,8 +36,7 @@ public final class FloatPrimitiveArraySerializer extends TypeSerializerSingleton private static final float[] EMPTY = new float[0]; public static final FloatPrimitiveArraySerializer INSTANCE = new FloatPrimitiveArraySerializer(); - - + @Override public boolean isImmutableType() { return false; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/IntPrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/IntPrimitiveArraySerializer.java index a5a3e568eba1b..f11b46eb893df 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/IntPrimitiveArraySerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/IntPrimitiveArraySerializer.java @@ -36,8 +36,7 @@ public class IntPrimitiveArraySerializer extends TypeSerializerSingleton{ private static final int[] EMPTY = new int[0]; public static final IntPrimitiveArraySerializer INSTANCE = new IntPrimitiveArraySerializer(); - - + @Override public boolean isImmutableType() { return false; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/LongPrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/LongPrimitiveArraySerializer.java index 3ecae3648b427..b0788c9d62b3c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/LongPrimitiveArraySerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/LongPrimitiveArraySerializer.java @@ -36,8 +36,7 @@ public final class LongPrimitiveArraySerializer extends TypeSerializerSingleton< private static final long[] EMPTY = new long[0]; public static final LongPrimitiveArraySerializer INSTANCE = new LongPrimitiveArraySerializer(); - - + @Override public boolean isImmutableType() { return false; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/ShortPrimitiveArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/ShortPrimitiveArraySerializer.java index 7f542c3bbf43b..107fe033567a4 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/ShortPrimitiveArraySerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/ShortPrimitiveArraySerializer.java @@ -36,8 +36,7 @@ public final class ShortPrimitiveArraySerializer extends TypeSerializerSingleton private static final short[] EMPTY = new short[0]; public static final ShortPrimitiveArraySerializer INSTANCE = new ShortPrimitiveArraySerializer(); - - + @Override public boolean isImmutableType() { return false; diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/StringArraySerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/StringArraySerializer.java index ccf369a7163e9..61a654c506756 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/StringArraySerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/array/StringArraySerializer.java @@ -38,8 +38,7 @@ public final class StringArraySerializer extends TypeSerializerSingleton extends TypeSerializer { private final Class type; private final Class typeToInstantiate; + + /** + * Map of class tag (using classname as tag) to their Kryo registration. + * + *

This map serves as a preview of the final registration result of + * the Kryo instance, taking into account registration overwrites. + */ + private LinkedHashMap kryoRegistrations; private transient ReflectDatumWriter writer; private transient ReflectDatumReader reader; @@ -73,6 +88,8 @@ public AvroSerializer(Class type, Class typeToInstantiate) { this.typeToInstantiate = checkNotNull(typeToInstantiate); InstantiationUtil.checkForInstantiation(typeToInstantiate); + + this.kryoRegistrations = buildKryoRegistrations(type); } // -------------------------------------------------------------------------------------------- @@ -165,14 +182,9 @@ private void checkKryoInitialized() { instantiatorStrategy.setFallbackInstantiatorStrategy(new StdInstantiatorStrategy()); kryo.setInstantiatorStrategy(instantiatorStrategy); - // register Avro types. - this.kryo.register(GenericData.Array.class, new Serializers.SpecificInstanceCollectionSerializerForArrayList()); - this.kryo.register(Utf8.class); - this.kryo.register(GenericData.EnumSymbol.class); - this.kryo.register(GenericData.Fixed.class); - this.kryo.register(GenericData.StringType.class); - this.kryo.setAsmEnabled(true); - this.kryo.register(type); + kryo.setAsmEnabled(true); + + KryoUtils.applyRegistrations(kryo, kryoRegistrations.values()); } } @@ -201,4 +213,120 @@ public boolean equals(Object obj) { public boolean canEqual(Object obj) { return obj instanceof AvroSerializer; } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public AvroSerializerConfigSnapshot snapshotConfiguration() { + return new AvroSerializerConfigSnapshot<>(type, typeToInstantiate, kryoRegistrations); + } + + @SuppressWarnings("unchecked") + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof AvroSerializerConfigSnapshot) { + final AvroSerializerConfigSnapshot config = (AvroSerializerConfigSnapshot) configSnapshot; + + if (type.equals(config.getTypeClass()) && typeToInstantiate.equals(config.getTypeToInstantiate())) { + // resolve Kryo registrations; currently, since the Kryo registrations in Avro + // are fixed, there shouldn't be a problem with the resolution here. + + LinkedHashMap oldRegistrations = config.getKryoRegistrations(); + oldRegistrations.putAll(kryoRegistrations); + + for (Map.Entry reconfiguredRegistrationEntry : kryoRegistrations.entrySet()) { + if (reconfiguredRegistrationEntry.getValue().isDummy()) { + return MigrationStrategy.migrate(); + } + } + + this.kryoRegistrations = oldRegistrations; + return MigrationStrategy.noMigration(); + } + } + + // ends up here if the preceding serializer is not + // the ValueSerializer, or serialized data type has changed + return MigrationStrategy.migrate(); + } + + public static class AvroSerializerConfigSnapshot extends KryoRegistrationSerializerConfigSnapshot { + + private static final int VERSION = 1; + + private Class typeToInstantiate; + + public AvroSerializerConfigSnapshot() {} + + public AvroSerializerConfigSnapshot( + Class baseType, + Class typeToInstantiate, + LinkedHashMap kryoRegistrations) { + + super(baseType, kryoRegistrations); + this.typeToInstantiate = Preconditions.checkNotNull(typeToInstantiate); + } + + @Override + public void write(DataOutputView out) throws IOException { + super.write(out); + + out.writeUTF(typeToInstantiate.getName()); + } + + @SuppressWarnings("unchecked") + @Override + public void read(DataInputView in) throws IOException { + super.read(in); + + String classname = in.readUTF(); + try { + typeToInstantiate = (Class) Class.forName(classname, true, getUserCodeClassLoader()); + } catch (ClassNotFoundException e) { + throw new IOException("Cannot find requested class " + classname + " in classpath.", e); + } + } + + @Override + public int getVersion() { + return VERSION; + } + + public Class getTypeToInstantiate() { + return typeToInstantiate; + } + } + + // -------------------------------------------------------------------------------------------- + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + + // kryoRegistrations may be null if this Avro serializer is deserialized from an old version + if (kryoRegistrations == null) { + this.kryoRegistrations = buildKryoRegistrations(type); + } + } + + private static LinkedHashMap buildKryoRegistrations(Class serializedDataType) { + final LinkedHashMap registrations = new LinkedHashMap<>(); + + // register Avro types. + registrations.put( + GenericData.Array.class.getName(), + new KryoRegistration( + GenericData.Array.class, + new ExecutionConfig.SerializableSerializer<>(new Serializers.SpecificInstanceCollectionSerializerForArrayList()))); + registrations.put(Utf8.class.getName(), new KryoRegistration(Utf8.class)); + registrations.put(GenericData.EnumSymbol.class.getName(), new KryoRegistration(GenericData.EnumSymbol.class)); + registrations.put(GenericData.Fixed.class.getName(), new KryoRegistration(GenericData.Fixed.class)); + registrations.put(GenericData.StringType.class.getName(), new KryoRegistration(GenericData.StringType.class)); + + // register the serialized data type + registrations.put(serializedDataType.getName(), new KryoRegistration(serializedDataType)); + + return registrations; + } } 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 1f3fcbc5a85ac..2a29a3f379207 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java @@ -21,7 +21,10 @@ import java.io.IOException; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.GenericTypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.types.CopyableValue; @@ -30,7 +33,7 @@ import static org.apache.flink.util.Preconditions.checkNotNull; @Internal -public class CopyableValueSerializer> extends TypeSerializer { +public final class CopyableValueSerializer> extends TypeSerializer { private static final long serialVersionUID = 1L; @@ -128,4 +131,41 @@ public boolean equals(Object obj) { public boolean canEqual(Object obj) { return obj instanceof CopyableValueSerializer; } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public CopyableValueSerializerConfigSnapshot snapshotConfiguration() { + return new CopyableValueSerializerConfigSnapshot<>(valueClass); + } + + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof CopyableValueSerializerConfigSnapshot + && valueClass.equals(((CopyableValueSerializerConfigSnapshot) configSnapshot).getTypeClass())) { + return MigrationStrategy.noMigration(); + } else { + return MigrationStrategy.migrate(); + } + } + + public static final class CopyableValueSerializerConfigSnapshot> + extends GenericTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public CopyableValueSerializerConfigSnapshot() {} + + public CopyableValueSerializerConfigSnapshot(Class copyableValueClass) { + super(copyableValueClass); + } + + @Override + public int getVersion() { + return VERSION; + } + } } 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 d9018da982c62..aad11f69c2f30 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java @@ -19,7 +19,9 @@ package org.apache.flink.api.java.typeutils.runtime; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.types.Either; @@ -183,4 +185,41 @@ public boolean canEqual(Object obj) { public int hashCode() { return 17 * leftSerializer.hashCode() + rightSerializer.hashCode(); } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public EitherSerializerConfigSnapshot snapshotConfiguration() { + return new EitherSerializerConfigSnapshot( + leftSerializer.snapshotConfiguration(), + rightSerializer.snapshotConfiguration()); + } + + @Override + protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof EitherSerializerConfigSnapshot) { + TypeSerializerConfigSnapshot[] leftRightSerializerConfigSnapshots = + ((EitherSerializerConfigSnapshot) configSnapshot).getNestedSerializerConfigSnapshots(); + + MigrationStrategy leftStrategy = leftSerializer.getMigrationStrategyFor(leftRightSerializerConfigSnapshots[0]); + MigrationStrategy rightStrategy = rightSerializer.getMigrationStrategyFor(leftRightSerializerConfigSnapshots[1]); + + if (leftStrategy.requireMigration() || rightStrategy.requireMigration()) { + if (leftStrategy.getFallbackDeserializer() != null && rightStrategy.getFallbackDeserializer() != null) { + return MigrationStrategy.migrateWithFallbackDeserializer( + new EitherSerializer<>( + leftStrategy.getFallbackDeserializer(), + rightStrategy.getFallbackDeserializer())); + } else { + return MigrationStrategy.migrate(); + } + } else { + return MigrationStrategy.noMigration(); + } + } else { + return MigrationStrategy.migrate(); + } + } } 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 new file mode 100644 index 0000000000000..473d4385dd15f --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializerConfigSnapshot.java @@ -0,0 +1,49 @@ +/* + * 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.java.typeutils.runtime; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +import org.apache.flink.types.Either; + +/** + * Configuration snapshot for serializers of the {@link Either} type, + * containing configuration snapshots of the Left and Right serializers. + */ +@Internal +public final class EitherSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public EitherSerializerConfigSnapshot() {} + + public EitherSerializerConfigSnapshot( + TypeSerializerConfigSnapshot leftSerializerConfigSnapshot, + TypeSerializerConfigSnapshot rightSerializerConfigSnapshot) { + + super(leftSerializerConfigSnapshot, rightSerializerConfigSnapshot); + } + + @Override + public int getVersion() { + return VERSION; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoRegistration.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoRegistration.java new file mode 100644 index 0000000000000..882073d6e1a4f --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoRegistration.java @@ -0,0 +1,173 @@ +/* + * 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.java.typeutils.runtime; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.factories.ReflectionSerializerFactory; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.typeutils.runtime.KryoRegistrationSerializerConfigSnapshot; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; + +/** + * A {@code KryoRegistration} resembles a registered class and its serializer in Kryo. + */ +@Internal +public class KryoRegistration implements Serializable { + + private static final long serialVersionUID = 5375110512910892655L; + + /** IMPORTANT: the order of the enumerations must not change, since their ordinals are used for serialization. */ + public enum SerializerDefinitionType { + UNSPECIFIED, CLASS, INSTANCE + } + + /** + * The registered class. + * + *

This can be a dummy class {@link KryoRegistrationSerializerConfigSnapshot.DummyRegisteredClass} if + * the class no longer exists when this registration instance was restored. + */ + private final Class registeredClass; + + /** + * Class of the serializer to use for the registered class. + * Exists only if the serializer definition type is {@link SerializerDefinitionType#CLASS}. + * + *

This can be a dummy serializer {@link KryoRegistrationSerializerConfigSnapshot.DummyKryoSerializerClass} if + * the serializer class no longer exists when this registration instance was restored. + */ + private final Class> serializerClass; + + /** + * A serializable instance of the serializer to use for the registered class. + * Exists only if the serializer definition type is {@link SerializerDefinitionType#INSTANCE}. + * + *

This can be a dummy serializer {@link KryoRegistrationSerializerConfigSnapshot.DummyKryoSerializerClass} if + * the serializer class no longer exists or is no longer valid when this registration instance was restored. + */ + private final ExecutionConfig.SerializableSerializer> serializableSerializerInstance; + + private SerializerDefinitionType serializerDefinitionType; + + public KryoRegistration(Class registeredClass) { + this.registeredClass = Preconditions.checkNotNull(registeredClass); + + this.serializerClass = null; + this.serializableSerializerInstance = null; + + this.serializerDefinitionType = SerializerDefinitionType.UNSPECIFIED; + } + + public KryoRegistration(Class registeredClass, Class> serializerClass) { + this.registeredClass = Preconditions.checkNotNull(registeredClass); + + this.serializerClass = Preconditions.checkNotNull(serializerClass); + this.serializableSerializerInstance = null; + + this.serializerDefinitionType = SerializerDefinitionType.CLASS; + } + + public KryoRegistration( + Class registeredClass, + ExecutionConfig.SerializableSerializer> serializableSerializerInstance) { + this.registeredClass = Preconditions.checkNotNull(registeredClass); + + this.serializerClass = null; + this.serializableSerializerInstance = Preconditions.checkNotNull(serializableSerializerInstance); + + this.serializerDefinitionType = SerializerDefinitionType.INSTANCE; + } + + public Class getRegisteredClass() { + return registeredClass; + } + + public SerializerDefinitionType getSerializerDefinitionType() { + return serializerDefinitionType; + } + + public Class> getSerializerClass() { + return serializerClass; + } + + public ExecutionConfig.SerializableSerializer> getSerializableSerializerInstance() { + return serializableSerializerInstance; + } + + public Serializer getSerializer(Kryo kryo) { + switch (serializerDefinitionType) { + case UNSPECIFIED: + return null; + case CLASS: + return ReflectionSerializerFactory.makeSerializer(kryo, serializerClass, registeredClass); + case INSTANCE: + return serializableSerializerInstance.getSerializer(); + default: + // this should not happen; adding as a guard for the future + throw new IllegalStateException( + "Unrecognized Kryo registration serializer definition type: " + serializerDefinitionType); + } + } + + public boolean isDummy() { + return registeredClass.equals(KryoRegistrationSerializerConfigSnapshot.DummyRegisteredClass.class) + || (serializerClass != null + && serializerClass.equals(KryoRegistrationSerializerConfigSnapshot.DummyKryoSerializerClass.class)) + || (serializableSerializerInstance != null + && serializableSerializerInstance.getSerializer() instanceof KryoRegistrationSerializerConfigSnapshot.DummyKryoSerializerClass); + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + + if (obj == null) { + return false; + } + + if (obj instanceof KryoRegistration) { + KryoRegistration other = (KryoRegistration) obj; + + // we cannot include the serializer instances here because they don't implement the equals method + return serializerDefinitionType == other.serializerDefinitionType + && registeredClass == other.registeredClass + && serializerClass == other.serializerClass; + } else { + return false; + } + } + + @Override + public int hashCode() { + int result = serializerDefinitionType.hashCode(); + result = 31 * result + registeredClass.hashCode(); + + if (serializerClass != null) { + result = 31 * result + serializerClass.hashCode(); + } + + return result; + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoRegistrationSerializerConfigSnapshot.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoRegistrationSerializerConfigSnapshot.java new file mode 100644 index 0000000000000..3a42d6912c818 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoRegistrationSerializerConfigSnapshot.java @@ -0,0 +1,251 @@ +/* + * 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.java.typeutils.runtime; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.GenericTypeSerializerConfigSnapshot; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.core.io.IOReadableWritable; +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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InvalidClassException; +import java.io.Serializable; +import java.util.LinkedHashMap; +import java.util.Map; + +/** + * Configuration snapshot base class for serializers that use Kryo for serialization. + * + *

The configuration captures the order of Kryo serializer registrations, so that new + * Kryo serializers can determine how to reconfigure their registration order to retain + * backwards compatibility. + * + * @param the data type that the Kryo serializer handles. + */ +@Internal +public abstract class KryoRegistrationSerializerConfigSnapshot extends GenericTypeSerializerConfigSnapshot { + + private static final Logger LOG = LoggerFactory.getLogger(KryoRegistrationSerializerConfigSnapshot.class); + + /** Map of class tag to the registration, with ordering. */ + private LinkedHashMap kryoRegistrations; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public KryoRegistrationSerializerConfigSnapshot() {} + + public KryoRegistrationSerializerConfigSnapshot( + Class typeClass, + LinkedHashMap kryoRegistrations) { + + super(typeClass); + + this.kryoRegistrations = Preconditions.checkNotNull(kryoRegistrations); + } + + @Override + public void write(DataOutputView out) throws IOException { + super.write(out); + + out.writeInt(kryoRegistrations.size()); + for (Map.Entry kryoRegistrationEntry : kryoRegistrations.entrySet()) { + out.writeUTF(kryoRegistrationEntry.getKey()); + new KryoRegistrationSerializationProxy<>(kryoRegistrationEntry.getValue()).write(out); + } + } + + @Override + public void read(DataInputView in) throws IOException { + super.read(in); + + int numKryoRegistrations = in.readInt(); + kryoRegistrations = new LinkedHashMap<>(numKryoRegistrations); + + KryoRegistrationSerializationProxy proxy; + for (int i = 0; i < numKryoRegistrations; i++) { + String classTag = in.readUTF(); + + proxy = new KryoRegistrationSerializationProxy(getUserCodeClassLoader()); + proxy.read(in); + + kryoRegistrations.put(classTag, proxy.kryoRegistration); + } + } + + public LinkedHashMap getKryoRegistrations() { + return kryoRegistrations; + } + + // -------------------------------------------------------------------------------------------- + + private static class KryoRegistrationSerializationProxy implements IOReadableWritable { + + private ClassLoader userCodeClassLoader; + + private KryoRegistration kryoRegistration; + + public KryoRegistrationSerializationProxy(ClassLoader userCodeClassLoader) { + this.userCodeClassLoader = Preconditions.checkNotNull(userCodeClassLoader); + } + + public KryoRegistrationSerializationProxy(KryoRegistration kryoRegistration) { + this.kryoRegistration = Preconditions.checkNotNull(kryoRegistration); + } + + @Override + public void write(DataOutputView out) throws IOException { + out.writeUTF(kryoRegistration.getRegisteredClass().getName()); + + final KryoRegistration.SerializerDefinitionType serializerDefinitionType = kryoRegistration.getSerializerDefinitionType(); + + out.writeInt(serializerDefinitionType.ordinal()); + switch (serializerDefinitionType) { + case UNSPECIFIED: + // nothing else to write + break; + case CLASS: + out.writeUTF(kryoRegistration.getSerializerClass().getName()); + break; + case INSTANCE: + InstantiationUtil.serializeObject(new DataOutputViewStream(out), kryoRegistration.getSerializableSerializerInstance()); + break; + default: + // this should not happen; adding as a guard for the future + throw new IllegalStateException( + "Unrecognized Kryo registration serializer definition type: " + serializerDefinitionType); + } + } + + @SuppressWarnings("unchecked") + @Override + public void read(DataInputView in) throws IOException { + String registeredClassname = in.readUTF(); + + Class registeredClass; + try { + registeredClass = (Class) Class.forName(registeredClassname, true, userCodeClassLoader); + } catch (ClassNotFoundException e) { + LOG.warn("Cannot find registered class " + registeredClassname + " for Kryo serialization in classpath;" + + " using a dummy class as a placeholder.", e); + + registeredClass = (Class) DummyRegisteredClass.class; + } + + final KryoRegistration.SerializerDefinitionType serializerDefinitionType = + KryoRegistration.SerializerDefinitionType.values()[in.readInt()]; + + switch (serializerDefinitionType) { + case UNSPECIFIED: + kryoRegistration = new KryoRegistration(registeredClass); + break; + + case CLASS: + String serializerClassname = in.readUTF(); + + Class serializerClass; + try { + serializerClass = Class.forName(serializerClassname, true, userCodeClassLoader); + } catch (ClassNotFoundException e) { + LOG.warn("Cannot find registered Kryo serializer class for class " + registeredClassname + + " in classpath; using a dummy Kryo serializer that should be replaced as soon as" + + " a new Kryo serializer for the class is present", e); + + serializerClass = DummyKryoSerializerClass.class; + } + + kryoRegistration = new KryoRegistration(registeredClass, serializerClass); + break; + + case INSTANCE: + ExecutionConfig.SerializableSerializer> serializerInstance; + try { + serializerInstance = InstantiationUtil.deserializeObject(new DataInputViewStream(in), userCodeClassLoader); + } catch (ClassNotFoundException e) { + LOG.warn("Cannot find registered Kryo serializer class for class " + registeredClassname + + " in classpath; using a dummy Kryo serializer that should be replaced as soon as" + + " a new Kryo serializer for the class is present", e); + + serializerInstance = new ExecutionConfig.SerializableSerializer<>(new DummyKryoSerializerClass()); + } catch (InvalidClassException e) { + LOG.warn("The registered Kryo serializer class for class " + registeredClassname + + " has changed and is no longer valid; using a dummy Kryo serializer that should be replaced" + + " as soon as a new Kryo serializer for the class is present.", e); + + serializerInstance = new ExecutionConfig.SerializableSerializer<>(new DummyKryoSerializerClass()); + } + + kryoRegistration = new KryoRegistration(registeredClass, serializerInstance); + break; + + default: + // this should not happen; adding as a guard for the future + throw new IllegalStateException( + "Unrecognized Kryo registration serializer definition type: " + serializerDefinitionType); + } + } + } + + /** + * Placeholder dummy for a previously registered class that can no longer be found in classpath on restore. + */ + public static class DummyRegisteredClass {} + + /** + * Placeholder dummmy for a previously registered Kryo serializer that is no longer valid or in classpath on restore. + */ + public static class DummyKryoSerializerClass extends Serializer implements Serializable { + + private static final long serialVersionUID = -6172780797425739308L; + + @Override + public void write(Kryo kryo, Output output, Object o) { + throw new UnsupportedOperationException( + "This exception indicates that you're trying to write a data type" + + " that no longer has a valid Kryo serializer registered for it."); + } + + @Override + public Object read(Kryo kryo, Input input, Class aClass) { + throw new UnsupportedOperationException( + "This exception indicates that you're trying to read a data type" + + " that no longer has a valid Kryo serializer registered for it."); + } + } + + @Override + public boolean equals(Object obj) { + return super.equals(obj) + && kryoRegistrations.equals(((KryoSerializer.KryoSerializerConfigSnapshot) obj).getKryoRegistrations()); + } + + @Override + public int hashCode() { + return super.hashCode() + kryoRegistrations.hashCode(); + } +} diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoUtils.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoUtils.java index 50c46e4d54767..0937ac7f30436 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoUtils.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/KryoUtils.java @@ -20,11 +20,13 @@ import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.KryoException; +import com.esotericsoftware.kryo.Serializer; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.util.InstantiationUtil; import java.io.IOException; +import java.util.Collection; /** * Convenience methods for Kryo @@ -86,4 +88,29 @@ public static T copy(T from, T reuse, Kryo kryo, TypeSerializer serialize } } } + + /** + * Apply a list of {@link KryoRegistration} to a Kryo instance. The list of registrations is + * assumed to already be a final resolution of all possible registration overwrites. + * + *

The registrations are applied in the given order and always specify the registration id as + * the next available id in the Kryo instance (providing the id just extra ensures nothing is + * overwritten, and isn't strictly required); + * + * @param kryo the Kryo instance to apply the registrations + * @param resolvedRegistrations the registrations, which should already be resolved of all possible registration overwrites + */ + public static void applyRegistrations(Kryo kryo, Collection resolvedRegistrations) { + + Serializer serializer; + for (KryoRegistration registration : resolvedRegistrations) { + serializer = registration.getSerializer(kryo); + + if (serializer != null) { + kryo.register(registration.getRegisteredClass(), serializer, kryo.getNextRegistrationId()); + } else { + kryo.register(registration.getRegisteredClass(), kryo.getNextRegistrationId()); + } + } + } } 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 1a9c8f992e768..2efdb2c815493 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 @@ -23,22 +23,25 @@ import java.io.ObjectOutputStream; import java.lang.reflect.Field; import java.lang.reflect.Modifier; -import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.LinkedHashSet; -import java.util.List; import java.util.Map; import java.util.Objects; import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.GenericTypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerUtil; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.Preconditions; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -53,22 +56,55 @@ public final class PojoSerializer extends TypeSerializer { private static final long serialVersionUID = 1L; - private final Class clazz; + // -------------------------------------------------------------------------------------------- + // PojoSerializer parameters + // -------------------------------------------------------------------------------------------- - private final TypeSerializer[] fieldSerializers; + /** The POJO type class. */ + private final Class clazz; + /** + * Fields of the POJO and their serializers. + * + *

The fields are kept as a separate transient member, with their serialization + * handled with the {@link #readObject(ObjectInputStream)} and {@link #writeObject(ObjectOutputStream)} + * methods. + * + *

These may be reconfigured in {@link #getMigrationStrategy(TypeSerializerConfigSnapshot)}. + */ + private transient Field[] fields; + private TypeSerializer[] fieldSerializers; private final int numFields; - private final Map, Integer> registeredClasses; - - private final TypeSerializer[] registeredSerializers; - + /** + * Registered subclasses and their serializers. + * Each subclass to their registered class tag is maintained as a separate map ordered by the class tag. + * + *

These may be reconfigured in {@link #getMigrationStrategy(TypeSerializerConfigSnapshot)}. + */ + private LinkedHashMap, Integer> registeredClasses; + private TypeSerializer[] registeredSerializers; + + /** + * Cache of non-registered subclasses to their serializers, created on-the-fly. + * + *

This cache is persisted and will be repopulated with reconfigured serializers + * in {@link #getMigrationStrategy(TypeSerializerConfigSnapshot)}. + */ + private transient HashMap, TypeSerializer> subclassSerializerCache; + + // -------------------------------------------------------------------------------------------- + + /** + * Configuration of the current execution. + * + *

Nested serializers created using this will have the most up-to-date configuration, + * and can be resolved for backwards compatibility with previous configuration + * snapshots in {@link #getMigrationStrategy(TypeSerializerConfigSnapshot)}. + */ private final ExecutionConfig executionConfig; - private transient Map, TypeSerializer> subclassSerializerCache; private transient ClassLoader cl; - // We need to handle these ourselves in writeObject()/readObject() - private transient Field[] fields; @SuppressWarnings("unchecked") public PojoSerializer( @@ -83,93 +119,20 @@ public PojoSerializer( this.numFields = fieldSerializers.length; this.executionConfig = checkNotNull(executionConfig); - LinkedHashSet> registeredPojoTypes = executionConfig.getRegisteredPojoTypes(); - for (int i = 0; i < numFields; i++) { this.fields[i].setAccessible(true); } cl = Thread.currentThread().getContextClassLoader(); - subclassSerializerCache = new HashMap, TypeSerializer>(); - // We only want those classes that are not our own class and are actually sub-classes. - List> cleanedTaggedClasses = new ArrayList>(registeredPojoTypes.size()); - for (Class registeredClass: registeredPojoTypes) { - if (registeredClass.equals(clazz)) { - continue; - } - if (!clazz.isAssignableFrom(registeredClass)) { - continue; - } - cleanedTaggedClasses.add(registeredClass); - - } - this.registeredClasses = new LinkedHashMap, Integer>(cleanedTaggedClasses.size()); - registeredSerializers = new TypeSerializer[cleanedTaggedClasses.size()]; - - int id = 0; - for (Class registeredClass: cleanedTaggedClasses) { - this.registeredClasses.put(registeredClass, id); - TypeInformation typeInfo = TypeExtractor.createTypeInfo(registeredClass); - registeredSerializers[id] = typeInfo.createSerializer(executionConfig); - - id++; - } - } - - private void writeObject(ObjectOutputStream out) - throws IOException, ClassNotFoundException { - out.defaultWriteObject(); - out.writeInt(fields.length); - for (Field field: fields) { - FieldSerializer.serializeField(field, out); - } - } - - private void readObject(ObjectInputStream in) - throws IOException, ClassNotFoundException { - in.defaultReadObject(); - int numFields = in.readInt(); - fields = new Field[numFields]; - for (int i = 0; i < numFields; i++) { - fields[i] = FieldSerializer.deserializeField(in); - } + LinkedHashSet> registeredSubclasses = + getRegisteredSubclassesFromExecutionConfig(clazz, executionConfig); - cl = Thread.currentThread().getContextClassLoader(); - subclassSerializerCache = new HashMap, TypeSerializer>(); - } + this.registeredClasses = createRegisteredSubclassTags(registeredSubclasses); + this.registeredSerializers = createRegisteredSubclassSerializers(registeredSubclasses, executionConfig); - private TypeSerializer getSubclassSerializer(Class subclass) { - TypeSerializer result = subclassSerializerCache.get(subclass); - if (result == null) { - - TypeInformation typeInfo = TypeExtractor.createTypeInfo(subclass); - result = typeInfo.createSerializer(executionConfig); - if (result instanceof PojoSerializer) { - PojoSerializer subclassSerializer = (PojoSerializer) result; - subclassSerializer.copyBaseFieldOrder(this); - } - subclassSerializerCache.put(subclass, result); - - } - return result; - } - - @SuppressWarnings("unused") - private boolean hasField(Field f) { - for (Field field: fields) { - if (f.equals(field)) { - return true; - } - } - return false; - } - - private void copyBaseFieldOrder(PojoSerializer baseSerializer) { - // do nothing for now, but in the future, adapt subclass serializer to have same - // ordering as base class serializer so that binary comparison on base class fields - // can work + this.subclassSerializerCache = new HashMap<>(); } @Override @@ -296,7 +259,7 @@ public T copy(T from, T reuse) { } } } catch (IllegalAccessException e) { - throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields" + "before."); + throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields before.", e); } } else { TypeSerializer subclassSerializer = getSubclassSerializer(actualType); @@ -341,13 +304,15 @@ public void serialize(T value, DataOutputView target) throws IOException { target.writeByte(flags); + // if its a registered subclass, write the class tag id, otherwise write the full classname if ((flags & IS_SUBCLASS) != 0) { target.writeUTF(actualClass.getName()); } else if ((flags & IS_TAGGED_SUBCLASS) != 0) { target.writeByte(subclassTag); } - + // if its a subclass, use the corresponding subclass serializer, + // otherwise serialize each field with our field serializers if ((flags & NO_SUBCLASS) != 0) { try { for (int i = 0; i < numFields; i++) { @@ -360,8 +325,7 @@ public void serialize(T value, DataOutputView target) throws IOException { } } } catch (IllegalAccessException e) { - throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields" + "before."); - + throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields before.", e); } } else { // subclass @@ -418,8 +382,7 @@ public T deserialize(DataInputView source) throws IOException { } } } catch (IllegalAccessException e) { - throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields" + "before."); - + throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields before.", e); } } else { if (subclassSerializer != null) { @@ -493,8 +456,7 @@ public T deserialize(T reuse, DataInputView source) throws IOException { } } } catch (IllegalAccessException e) { - throw new RuntimeException( - "Error during POJO copy, this should not happen since we check the fields before."); + throw new RuntimeException("Error during POJO copy, this should not happen since we check the fields before.", e); } } else { if (subclassSerializer != null) { @@ -574,4 +536,522 @@ public boolean equals(Object obj) { public boolean canEqual(Object obj) { return obj instanceof PojoSerializer; } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public PojoSerializerConfigSnapshot snapshotConfiguration() { + return buildConfigSnapshot( + clazz, + registeredClasses, + registeredSerializers, + fields, + fieldSerializers, + subclassSerializerCache); + } + + @SuppressWarnings("unchecked") + @Override + public MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof PojoSerializerConfigSnapshot) { + final PojoSerializerConfigSnapshot config = (PojoSerializerConfigSnapshot) configSnapshot; + + if (clazz.equals(config.getTypeClass())) { + if (this.numFields == config.getFieldToSerializerConfigSnapshot().size()) { + + MigrationStrategy strategy; + + // ----------- check field order and migration requirement of field serializers ----------- + + // reordered fields and their serializers; + // this won't be applied to this serializer until all migration checks have been completed + final Field[] reorderedFields = new Field[this.numFields]; + final TypeSerializer[] reorderedFieldSerializers = + (TypeSerializer[]) new TypeSerializer[this.numFields]; + + int i = 0; + for (Map.Entry fieldToConfigSnapshotEntry + : config.getFieldToSerializerConfigSnapshot().entrySet()) { + + int fieldIndex = findField(fieldToConfigSnapshotEntry.getKey()); + if (fieldIndex != -1) { + reorderedFields[i] = fieldToConfigSnapshotEntry.getKey(); + + strategy = fieldSerializers[fieldIndex].getMigrationStrategyFor(fieldToConfigSnapshotEntry.getValue()); + if (strategy.requireMigration()) { + return MigrationStrategy.migrate(); + } else { + reorderedFieldSerializers[i] = fieldSerializers[fieldIndex]; + } + } else { + return MigrationStrategy.migrate(); + } + + i++; + } + + // ---- check subclass registration order and migration requirement of registered serializers ---- + + // reordered subclass registrations and their serializers; + // this won't be applied to this serializer until all migration checks have been completed + final LinkedHashMap, Integer> reorderedRegisteredSubclassesToClasstags; + final TypeSerializer[] reorderedRegisteredSubclassSerializers; + + final LinkedHashMap, TypeSerializerConfigSnapshot> previousRegistrations = + config.getRegisteredSubclassesToSerializerConfigSnapshots(); + + // the reconfigured list of registered subclasses will be the previous registered + // subclasses in the original order with new subclasses appended at the end + LinkedHashSet> reorderedRegisteredSubclasses = new LinkedHashSet<>(); + reorderedRegisteredSubclasses.addAll(previousRegistrations.keySet()); + reorderedRegisteredSubclasses.addAll( + getRegisteredSubclassesFromExecutionConfig(clazz, executionConfig)); + + // restablish the registered class tags and serializers + reorderedRegisteredSubclassesToClasstags = createRegisteredSubclassTags(reorderedRegisteredSubclasses); + reorderedRegisteredSubclassSerializers = createRegisteredSubclassSerializers( + reorderedRegisteredSubclasses, executionConfig); + + i = 0; + for (TypeSerializerConfigSnapshot previousRegisteredSerializerConfig : previousRegistrations.values()) { + // check migration requirement of subclass serializer + strategy = reorderedRegisteredSubclassSerializers[i].getMigrationStrategyFor(previousRegisteredSerializerConfig); + if (strategy.requireMigration()) { + return MigrationStrategy.migrate(); + } + + i++; + } + + // ------------------ reconfigure non-registered subclass serializers ------------------ + + // the rebuilt cache for non-registered subclass serializers; + // this won't be applied to this serializer until all migration checks have been completed + HashMap, TypeSerializer> rebuiltCache = new HashMap<>(); + + for (Map.Entry, TypeSerializerConfigSnapshot> previousCachedEntry + : config.getNonRegisteredSubclassesToSerializerConfigSnapshots().entrySet()) { + + // check migration requirement of cached subclass serializer + TypeSerializer cachedSerializer = createSubclassSerializer(previousCachedEntry.getKey()); + + strategy = cachedSerializer.getMigrationStrategyFor(previousCachedEntry.getValue()); + if (strategy.requireMigration()) { + return MigrationStrategy.migrate(); + } else { + rebuiltCache.put(previousCachedEntry.getKey(), cachedSerializer); + } + } + + // completed migration checks; up to this point, we can just reconfigure + // the serializer so that migration is not required + + this.fields = reorderedFields; + this.fieldSerializers = reorderedFieldSerializers; + + this.registeredClasses = reorderedRegisteredSubclassesToClasstags; + this.registeredSerializers = reorderedRegisteredSubclassSerializers; + + this.subclassSerializerCache = rebuiltCache; + + return MigrationStrategy.noMigration(); + } + } + } + + return MigrationStrategy.migrate(); + } + + public static final class PojoSerializerConfigSnapshot extends GenericTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** + * Ordered map of POJO fields to the configuration snapshots of their corresponding serializers. + * + *

Ordering of the fields is kept so that new Pojo serializers for previous data + * may reorder the fields in case they are different. The order of the fields need to + * stay the same for binary compatibility, as the field order is part of the serialization format. + */ + private LinkedHashMap fieldToSerializerConfigSnapshot; + + /** + * Ordered map of registered subclasses to the configuration snapshots of their corresponding serializers. + * + *

Ordering of the registered subclasses is kept so that new Pojo serializers for previous data + * may retain the same class tag used for registered subclasses. Newly registered subclasses that + * weren't present before should be appended with the next available class tag. + */ + private LinkedHashMap, TypeSerializerConfigSnapshot> registeredSubclassesToSerializerConfigSnapshots; + + /** + * Configuration snapshots of previously cached non-registered subclass serializers. + * + *

This is kept so that new Pojo serializers may eagerly repopulate their + * cache with reconfigured subclass serializers. + */ + private HashMap, TypeSerializerConfigSnapshot> nonRegisteredSubclassesToSerializerConfigSnapshots; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public PojoSerializerConfigSnapshot() {} + + public PojoSerializerConfigSnapshot( + Class pojoType, + LinkedHashMap fieldToSerializerConfigSnapshot, + LinkedHashMap, TypeSerializerConfigSnapshot> registeredSubclassesToSerializerConfigSnapshots, + HashMap, TypeSerializerConfigSnapshot> nonRegisteredSubclassesToSerializerConfigSnapshots) { + + super(pojoType); + + this.fieldToSerializerConfigSnapshot = + Preconditions.checkNotNull(fieldToSerializerConfigSnapshot); + this.registeredSubclassesToSerializerConfigSnapshots = + Preconditions.checkNotNull(registeredSubclassesToSerializerConfigSnapshots); + this.nonRegisteredSubclassesToSerializerConfigSnapshots = + Preconditions.checkNotNull(nonRegisteredSubclassesToSerializerConfigSnapshots); + } + + @Override + public void write(DataOutputView out) throws IOException { + super.write(out); + + // --- write fields and their serializers, in order + + out.writeInt(fieldToSerializerConfigSnapshot.size()); + for (Map.Entry entry + : fieldToSerializerConfigSnapshot.entrySet()) { + out.writeUTF(entry.getKey().getName()); + TypeSerializerUtil.writeSerializerConfigSnapshot(out, entry.getValue()); + } + + // --- write registered subclasses and their serializers, in registration order + + out.writeInt(registeredSubclassesToSerializerConfigSnapshots.size()); + for (Map.Entry, TypeSerializerConfigSnapshot> entry + : registeredSubclassesToSerializerConfigSnapshots.entrySet()) { + out.writeUTF(entry.getKey().getName()); + TypeSerializerUtil.writeSerializerConfigSnapshot(out, entry.getValue()); + } + + // --- write snapshot of non-registered subclass serializer cache + + out.writeInt(nonRegisteredSubclassesToSerializerConfigSnapshots.size()); + for (Map.Entry, TypeSerializerConfigSnapshot> entry + : nonRegisteredSubclassesToSerializerConfigSnapshots.entrySet()) { + out.writeUTF(entry.getKey().getName()); + TypeSerializerUtil.writeSerializerConfigSnapshot(out, entry.getValue()); + } + } + + @Override + public void read(DataInputView in) throws IOException { + super.read(in); + + // --- read fields and their serializers, in order + + int numFields = in.readInt(); + this.fieldToSerializerConfigSnapshot = new LinkedHashMap<>(numFields); + String fieldName; + Field field; + for (int i = 0; i < numFields; i++) { + fieldName = in.readUTF(); + + // search all superclasses for the field + Class clazz = getTypeClass(); + field = null; + while (clazz != null) { + try { + field = clazz.getDeclaredField(fieldName); + field.setAccessible(true); + break; + } catch (NoSuchFieldException e) { + clazz = clazz.getSuperclass(); + } + } + + if (field == null) { + // the field no longer exists in the POJO + throw new IOException("Can't find field " + fieldName + " in POJO class " + getTypeClass().getName()); + } else { + fieldToSerializerConfigSnapshot.put( + field, + TypeSerializerUtil.readSerializerConfigSnapshot(in, getUserCodeClassLoader())); + } + } + + // --- read registered subclasses and their serializers, in registration order + + int numRegisteredSubclasses = in.readInt(); + this.registeredSubclassesToSerializerConfigSnapshots = new LinkedHashMap<>(numRegisteredSubclasses); + String registeredSubclassname; + Class registeredSubclass; + for (int i = 0; i < numRegisteredSubclasses; i++) { + registeredSubclassname = in.readUTF(); + try { + registeredSubclass = Class.forName(registeredSubclassname, true, getUserCodeClassLoader()); + } catch (ClassNotFoundException e) { + throw new IOException("Cannot find requested class " + registeredSubclassname + " in classpath.", e); + } + + this.registeredSubclassesToSerializerConfigSnapshots.put( + registeredSubclass, + TypeSerializerUtil.readSerializerConfigSnapshot(in, getUserCodeClassLoader())); + } + + // --- read snapshot of non-registered subclass serializer cache + + int numCachedSubclassSerializers = in.readInt(); + this.nonRegisteredSubclassesToSerializerConfigSnapshots = new HashMap<>(numCachedSubclassSerializers); + String cachedSubclassname; + Class cachedSubclass; + for (int i = 0; i < numCachedSubclassSerializers; i++) { + cachedSubclassname = in.readUTF(); + try { + cachedSubclass = Class.forName(cachedSubclassname, true, getUserCodeClassLoader()); + } catch (ClassNotFoundException e) { + throw new IOException("Cannot find requested class " + cachedSubclassname + " in classpath.", e); + } + + this.nonRegisteredSubclassesToSerializerConfigSnapshots.put( + cachedSubclass, + TypeSerializerUtil.readSerializerConfigSnapshot(in, getUserCodeClassLoader())); + } + } + + @Override + public int getVersion() { + return VERSION; + } + + public LinkedHashMap getFieldToSerializerConfigSnapshot() { + return fieldToSerializerConfigSnapshot; + } + + public LinkedHashMap, TypeSerializerConfigSnapshot> getRegisteredSubclassesToSerializerConfigSnapshots() { + return registeredSubclassesToSerializerConfigSnapshots; + } + + public HashMap, TypeSerializerConfigSnapshot> getNonRegisteredSubclassesToSerializerConfigSnapshots() { + return nonRegisteredSubclassesToSerializerConfigSnapshots; + } + + @Override + public boolean equals(Object obj) { + return super.equals(obj) + && (obj instanceof PojoSerializerConfigSnapshot) + && fieldToSerializerConfigSnapshot.equals(((PojoSerializerConfigSnapshot) obj).getFieldToSerializerConfigSnapshot()) + && registeredSubclassesToSerializerConfigSnapshots.equals(((PojoSerializerConfigSnapshot) obj).getRegisteredSubclassesToSerializerConfigSnapshots()) + && nonRegisteredSubclassesToSerializerConfigSnapshots.equals(((PojoSerializerConfigSnapshot) obj).nonRegisteredSubclassesToSerializerConfigSnapshots); + } + + @Override + public int hashCode() { + return super.hashCode() + + Objects.hash( + fieldToSerializerConfigSnapshot, + registeredSubclassesToSerializerConfigSnapshots, + nonRegisteredSubclassesToSerializerConfigSnapshots); + } + } + + // -------------------------------------------------------------------------------------------- + + private void writeObject(ObjectOutputStream out) + throws IOException, ClassNotFoundException { + out.defaultWriteObject(); + out.writeInt(fields.length); + for (Field field: fields) { + FieldSerializer.serializeField(field, out); + } + } + + private void readObject(ObjectInputStream in) + throws IOException, ClassNotFoundException { + in.defaultReadObject(); + int numFields = in.readInt(); + fields = new Field[numFields]; + for (int i = 0; i < numFields; i++) { + fields[i] = FieldSerializer.deserializeField(in); + } + + cl = Thread.currentThread().getContextClassLoader(); + subclassSerializerCache = new HashMap, TypeSerializer>(); + } + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + /** + * Extracts the subclasses of the base POJO class registered in the execution config. + */ + private static LinkedHashSet> getRegisteredSubclassesFromExecutionConfig( + Class basePojoClass, + ExecutionConfig executionConfig) { + + LinkedHashSet> subclassesInRegistrationOrder = new LinkedHashSet<>(executionConfig.getRegisteredPojoTypes().size()); + for (Class registeredClass : executionConfig.getRegisteredPojoTypes()) { + if (registeredClass.equals(basePojoClass)) { + continue; + } + if (!basePojoClass.isAssignableFrom(registeredClass)) { + continue; + } + subclassesInRegistrationOrder.add(registeredClass); + } + + return subclassesInRegistrationOrder; + } + + /** + * Builds map of registered subclasses to their class tags. + * Class tags will be integers starting from 0, assigned incrementally with the order of provided subclasses. + */ + private static LinkedHashMap, Integer> createRegisteredSubclassTags(LinkedHashSet> registeredSubclasses) { + final LinkedHashMap, Integer> classToTag = new LinkedHashMap<>(); + + int id = 0; + for (Class registeredClass : registeredSubclasses) { + classToTag.put(registeredClass, id); + id ++; + } + + return classToTag; + } + + /** + * Creates an array of serializers for provided list of registered subclasses. + * Order of returned serializers will correspond to order of provided subclasses. + */ + private static TypeSerializer[] createRegisteredSubclassSerializers( + LinkedHashSet> registeredSubclasses, + ExecutionConfig executionConfig) { + + final TypeSerializer[] subclassSerializers = new TypeSerializer[registeredSubclasses.size()]; + + int i = 0; + for (Class registeredClass : registeredSubclasses) { + subclassSerializers[i] = TypeExtractor.createTypeInfo(registeredClass).createSerializer(executionConfig); + i++; + } + + return subclassSerializers; + } + + /** + * Fetches cached serializer for a non-registered subclass; + * also creates the serializer if it doesn't exist yet. + * + * This method is also exposed to package-private access + * for testing purposes. + */ + TypeSerializer getSubclassSerializer(Class subclass) { + TypeSerializer result = subclassSerializerCache.get(subclass); + if (result == null) { + result = createSubclassSerializer(subclass); + subclassSerializerCache.put(subclass, result); + } + return result; + } + + private TypeSerializer createSubclassSerializer(Class subclass) { + TypeSerializer serializer = TypeExtractor.createTypeInfo(subclass).createSerializer(executionConfig); + + if (serializer instanceof PojoSerializer) { + PojoSerializer subclassSerializer = (PojoSerializer) serializer; + subclassSerializer.copyBaseFieldOrder(this); + } + + return serializer; + } + + /** + * Finds and returns the order (0-based) of a POJO field. + * Returns -1 if the field does not exist for this POJO. + */ + private int findField(Field f) { + int foundIndex = 0; + for (Field field : fields) { + if (f.equals(field)) { + return foundIndex; + } + + foundIndex++; + } + + return -1; + } + + private void copyBaseFieldOrder(PojoSerializer baseSerializer) { + // do nothing for now, but in the future, adapt subclass serializer to have same + // ordering as base class serializer so that binary comparison on base class fields + // can work + } + + /** + * Build and return a snapshot of the serializer's parameters and currently cached serializers. + */ + private static PojoSerializerConfigSnapshot buildConfigSnapshot( + Class pojoType, + LinkedHashMap, Integer> registeredSubclassesToTags, + TypeSerializer[] registeredSubclassSerializers, + Field[] fields, + TypeSerializer[] fieldSerializers, + HashMap, TypeSerializer> nonRegisteredSubclassSerializerCache) { + + final LinkedHashMap fieldToSerializerConfigSnapshots = + new LinkedHashMap<>(fields.length); + + for (int i = 0; i < fields.length; i++) { + fieldToSerializerConfigSnapshots.put(fields[i], fieldSerializers[i].snapshotConfiguration()); + } + + final LinkedHashMap, TypeSerializerConfigSnapshot> registeredSubclassesToSerializerConfigSnapshots = + new LinkedHashMap<>(registeredSubclassesToTags.size()); + + for (Map.Entry, Integer> entry : registeredSubclassesToTags.entrySet()) { + registeredSubclassesToSerializerConfigSnapshots.put( + entry.getKey(), + registeredSubclassSerializers[entry.getValue()].snapshotConfiguration()); + } + + final HashMap, TypeSerializerConfigSnapshot> nonRegisteredSubclassesToSerializerConfigSnapshots = + new LinkedHashMap<>(nonRegisteredSubclassSerializerCache.size()); + + for (Map.Entry, TypeSerializer> entry : nonRegisteredSubclassSerializerCache.entrySet()) { + nonRegisteredSubclassesToSerializerConfigSnapshots.put(entry.getKey(), entry.getValue().snapshotConfiguration()); + } + + return new PojoSerializerConfigSnapshot<>( + pojoType, + fieldToSerializerConfigSnapshots, + registeredSubclassesToSerializerConfigSnapshots, + nonRegisteredSubclassesToSerializerConfigSnapshots); + } + + // -------------------------------------------------------------------------------------------- + // Test utilities + // -------------------------------------------------------------------------------------------- + + @VisibleForTesting + Field[] getFields() { + return fields; + } + + @VisibleForTesting + LinkedHashMap, Integer> getRegisteredClasses() { + return registeredClasses; + } + + @VisibleForTesting + TypeSerializer[] getRegisteredSerializers() { + return registeredSerializers; + } + + @VisibleForTesting + HashMap, TypeSerializer> getSubclassSerializerCache() { + return subclassSerializerCache; + } } 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 dbd5d3a549925..1ffd9fc8dabbb 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java @@ -18,12 +18,17 @@ package org.apache.flink.api.java.typeutils.runtime; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerUtil; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.types.Row; import java.io.IOException; +import java.io.ObjectInputStream; import java.util.Arrays; import static org.apache.flink.api.java.typeutils.runtime.NullMaskUtils.readIntoAndCopyNullMask; @@ -35,12 +40,15 @@ * Serializer for {@link Row}. */ @Internal -public class RowSerializer extends TypeSerializer { +public final class RowSerializer extends TypeSerializer { private static final long serialVersionUID = 1L; - private final boolean[] nullMask; + private final TypeSerializer[] fieldSerializers; + private transient boolean[] nullMask; + + @SuppressWarnings("unchecked") public RowSerializer(TypeSerializer[] fieldSerializers) { this.fieldSerializers = (TypeSerializer[]) checkNotNull(fieldSerializers); this.nullMask = new boolean[fieldSerializers.length]; @@ -231,4 +239,73 @@ public boolean canEqual(Object obj) { public int hashCode() { return Arrays.hashCode(fieldSerializers); } + + // -------------------------------------------------------------------------------------------- + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + this.nullMask = new boolean[fieldSerializers.length]; + } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public RowSerializerConfigSnapshot snapshotConfiguration() { + return new RowSerializerConfigSnapshot(TypeSerializerUtil.snapshotConfigurations(fieldSerializers)); + } + + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof RowSerializerConfigSnapshot) { + TypeSerializerConfigSnapshot[] fieldSerializerConfigSnapshots = + ((RowSerializerConfigSnapshot) configSnapshot).getNestedSerializerConfigSnapshots(); + + if (fieldSerializerConfigSnapshots.length == fieldSerializers.length) { + boolean requireMigration = false; + TypeSerializer[] fallbackFieldSerializers = new TypeSerializer[fieldSerializers.length]; + + MigrationStrategy strategy; + for (int i = 0; i < fieldSerializers.length; i++) { + strategy = fieldSerializers[i].getMigrationStrategyFor(fieldSerializerConfigSnapshots[i]); + if (strategy.requireMigration()) { + requireMigration = true; + + if (strategy.getFallbackDeserializer() == null) { + // one of the field serializers cannot provide a fallback deserializer + return MigrationStrategy.migrate(); + } else { + fallbackFieldSerializers[i] = strategy.getFallbackDeserializer(); + } + } + } + + if (requireMigration) { + return MigrationStrategy.migrateWithFallbackDeserializer(new RowSerializer(fallbackFieldSerializers)); + } else { + return MigrationStrategy.noMigration(); + } + } + } + + return MigrationStrategy.migrate(); + } + + public static final class RowSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public RowSerializerConfigSnapshot() {} + + public RowSerializerConfigSnapshot(TypeSerializerConfigSnapshot[] fieldSerializerConfigSnapshots) { + super(fieldSerializerConfigSnapshots); + } + + @Override + public int getVersion() { + return VERSION; + } + } } 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 afc4aa24aa1c8..ca5c23c4dcada 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 @@ -19,7 +19,10 @@ package org.apache.flink.api.java.typeutils.runtime; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerUtil; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -115,4 +118,43 @@ public boolean equals(Object obj) { public boolean canEqual(Object obj) { return obj instanceof TupleSerializerBase; } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public TupleSerializerConfigSnapshot snapshotConfiguration() { + return new TupleSerializerConfigSnapshot<>( + tupleClass, + TypeSerializerUtil.snapshotConfigurations(fieldSerializers)); + } + + @SuppressWarnings("unchecked") + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof TupleSerializerConfigSnapshot) { + final TupleSerializerConfigSnapshot config = (TupleSerializerConfigSnapshot) configSnapshot; + + if (tupleClass.equals(config.getTupleClass())) { + TypeSerializerConfigSnapshot[] fieldSerializerConfigSnapshots = + ((TupleSerializerConfigSnapshot) configSnapshot).getNestedSerializerConfigSnapshots(); + + if (fieldSerializerConfigSnapshots.length == fieldSerializers.length) { + + MigrationStrategy strategy; + for (int i = 0; i < fieldSerializers.length; i++) { + strategy = fieldSerializers[i].getMigrationStrategyFor(fieldSerializerConfigSnapshots[i]); + if (strategy.requireMigration()) { + return MigrationStrategy.migrate(); + } + } + + return MigrationStrategy.noMigration(); + } + } + } + + return MigrationStrategy.migrate(); + } } 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 new file mode 100644 index 0000000000000..6d2bb5f201915 --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/TupleSerializerConfigSnapshot.java @@ -0,0 +1,91 @@ +/* + * 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.java.typeutils.runtime; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +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; + +/** + * Snapshot of a tuple serializer's configuration. + */ +@Internal +public final class TupleSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + private Class tupleClass; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public TupleSerializerConfigSnapshot() {} + + public TupleSerializerConfigSnapshot( + Class tupleClass, + TypeSerializerConfigSnapshot[] fieldSerializerConfigSnapshots) { + + super(fieldSerializerConfigSnapshots); + + this.tupleClass = Preconditions.checkNotNull(tupleClass); + } + + @Override + public void write(DataOutputView out) throws IOException { + super.write(out); + + InstantiationUtil.serializeObject(new DataOutputViewStream(out), tupleClass); + } + + @Override + public void read(DataInputView in) throws IOException { + super.read(in); + + try { + tupleClass = InstantiationUtil.deserializeObject(new DataInputViewStream(in), getUserCodeClassLoader()); + } catch (ClassNotFoundException e) { + throw new IOException("Could not find requested tuple class in classpath.", e); + } + } + + @Override + public int getVersion() { + return VERSION; + } + + public Class getTupleClass() { + return tupleClass; + } + + @Override + public boolean equals(Object obj) { + return super.equals(obj) + && (obj instanceof TupleSerializerConfigSnapshot) + && (tupleClass.equals(((TupleSerializerConfigSnapshot) obj).getTupleClass())); + } + + @Override + public int hashCode() { + return super.hashCode() * 31 + tupleClass.hashCode(); + } +} 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 56e204c9cecb8..4ca8bcc595eee 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 @@ -19,15 +19,20 @@ package org.apache.flink.api.java.typeutils.runtime; import java.io.IOException; +import java.io.ObjectInputStream; +import java.util.LinkedHashMap; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.types.Value; import org.apache.flink.util.InstantiationUtil; import com.esotericsoftware.kryo.Kryo; +import org.apache.flink.util.Preconditions; import org.objenesis.strategy.StdInstantiatorStrategy; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -39,12 +44,23 @@ * @param The type serialized. */ @Internal -public class ValueSerializer extends TypeSerializer { +public final class ValueSerializer extends TypeSerializer { private static final long serialVersionUID = 1L; private final Class type; - + + /** + * Map of class tag (using classname as tag) to their Kryo registration. + * + *

This map serves as a preview of the final registration result of + * the Kryo instance, taking into account registration overwrites. + * + *

Currently, we only have one single registration for the value type. + * Nevertheless, we keep this information here for future compatibility. + */ + private LinkedHashMap kryoRegistrations; + private transient Kryo kryo; private transient T copyInstance; @@ -53,6 +69,7 @@ public class ValueSerializer extends TypeSerializer { public ValueSerializer(Class type) { this.type = checkNotNull(type); + this.kryoRegistrations = asKryoRegistrations(type); } // -------------------------------------------------------------------------------------------- @@ -126,7 +143,8 @@ private void checkKryoInitialized() { kryo.setInstantiatorStrategy(instantiatorStrategy); this.kryo.setAsmEnabled(true); - this.kryo.register(type); + + KryoUtils.applyRegistrations(this.kryo, kryoRegistrations.values()); } } @@ -152,4 +170,66 @@ public boolean equals(Object obj) { public boolean canEqual(Object obj) { return obj instanceof ValueSerializer; } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public ValueSerializerConfigSnapshot snapshotConfiguration() { + return new ValueSerializerConfigSnapshot<>(type); + } + + @SuppressWarnings("unchecked") + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof ValueSerializerConfigSnapshot) { + final ValueSerializerConfigSnapshot config = (ValueSerializerConfigSnapshot) configSnapshot; + + if (type.equals(config.getTypeClass())) { + // currently, simply checking the type of the value class is sufficient; + // in the future, if there are more Kryo registrations, we should try to resolve that + return MigrationStrategy.noMigration(); + } + } + + return MigrationStrategy.migrate(); + } + + public static class ValueSerializerConfigSnapshot extends KryoRegistrationSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public ValueSerializerConfigSnapshot() {} + + public ValueSerializerConfigSnapshot(Class valueTypeClass) { + super(valueTypeClass, asKryoRegistrations(valueTypeClass)); + } + + @Override + public int getVersion() { + return VERSION; + } + } + + // -------------------------------------------------------------------------------------------- + + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + + // kryoRegistrations may be null if this value serializer is deserialized from an old version + if (kryoRegistrations == null) { + this.kryoRegistrations = asKryoRegistrations(type); + } + } + + private static LinkedHashMap asKryoRegistrations(Class type) { + Preconditions.checkNotNull(type); + + LinkedHashMap registration = new LinkedHashMap<>(1); + registration.put(type.getClass().getName(), new KryoRegistration(type)); + + return registration; + } } 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 cba0c8415f08e..a0cf99cd8443f 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 @@ -21,18 +21,22 @@ import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.KryoException; import com.esotericsoftware.kryo.Serializer; -import com.esotericsoftware.kryo.factories.ReflectionSerializerFactory; import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; import org.apache.avro.generic.GenericData; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream; import org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream; +import org.apache.flink.api.java.typeutils.runtime.KryoRegistration; +import org.apache.flink.api.java.typeutils.runtime.KryoRegistrationSerializerConfigSnapshot; +import org.apache.flink.api.java.typeutils.runtime.KryoUtils; import org.apache.flink.api.java.typeutils.runtime.NoFetchingInput; -import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers.SpecificInstanceCollectionSerializerForArrayList; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -45,6 +49,7 @@ import java.io.ByteArrayOutputStream; import java.io.EOFException; import java.io.IOException; +import java.io.ObjectInputStream; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.lang.reflect.Modifier; @@ -72,11 +77,16 @@ public class KryoSerializer extends TypeSerializer { // ------------------------------------------------------------------------ - private final LinkedHashMap, ExecutionConfig.SerializableSerializer> registeredTypesWithSerializers; - private final LinkedHashMap, Class>> registeredTypesWithSerializerClasses; private final LinkedHashMap, ExecutionConfig.SerializableSerializer> defaultSerializers; private final LinkedHashMap, Class>> defaultSerializerClasses; - private final LinkedHashSet> registeredTypes; + + /** + * Map of class tag (using classname as tag) to their Kryo registration. + * + *

This map serves as a preview of the final registration result of + * the Kryo instance, taking into account registration overwrites. + */ + private LinkedHashMap kryoRegistrations; private final Class type; @@ -92,6 +102,13 @@ public class KryoSerializer extends TypeSerializer { private transient Input input; private transient Output output; + // ------------------------------------------------------------------------ + // legacy fields; these fields cannot yet be removed to retain backwards compatibility + + private LinkedHashMap, ExecutionConfig.SerializableSerializer> registeredTypesWithSerializers; + private LinkedHashMap, Class>> registeredTypesWithSerializerClasses; + private LinkedHashSet> registeredTypes; + // ------------------------------------------------------------------------ public KryoSerializer(Class type, ExecutionConfig executionConfig){ @@ -99,20 +116,22 @@ public KryoSerializer(Class type, ExecutionConfig executionConfig){ this.defaultSerializers = executionConfig.getDefaultKryoSerializers(); this.defaultSerializerClasses = executionConfig.getDefaultKryoSerializerClasses(); - this.registeredTypesWithSerializers = executionConfig.getRegisteredTypesWithKryoSerializers(); - this.registeredTypesWithSerializerClasses = executionConfig.getRegisteredTypesWithKryoSerializerClasses(); - this.registeredTypes = executionConfig.getRegisteredKryoTypes(); + + this.kryoRegistrations = buildKryoRegistrations( + this.type, + executionConfig.getRegisteredKryoTypes(), + executionConfig.getRegisteredTypesWithKryoSerializerClasses(), + executionConfig.getRegisteredTypesWithKryoSerializers()); } /** * Copy-constructor that does not copy transient fields. They will be initialized once required. */ protected KryoSerializer(KryoSerializer toCopy) { - registeredTypesWithSerializers = toCopy.registeredTypesWithSerializers; - registeredTypesWithSerializerClasses = toCopy.registeredTypesWithSerializerClasses; defaultSerializers = toCopy.defaultSerializers; defaultSerializerClasses = toCopy.defaultSerializerClasses; - registeredTypes = toCopy.registeredTypes; + + kryoRegistrations = toCopy.kryoRegistrations; type = toCopy.type; if(type == null){ @@ -255,6 +274,21 @@ public void copy(DataInputView source, DataOutputView target) throws IOException T tmp = deserialize(copyInstance, source); serialize(tmp, target); } + + @Override + public boolean canRestoreFrom(TypeSerializer other) { + if (other instanceof KryoSerializer) { + KryoSerializer otherKryo = (KryoSerializer) other; + + // we cannot include the Serializers here because they don't implement the equals method + return other.canEqual(this) && + type == otherKryo.type && + (kryoRegistrations.equals(otherKryo.kryoRegistrations)) && + (defaultSerializerClasses.equals(otherKryo.defaultSerializerClasses) || otherKryo.defaultSerializerClasses.isEmpty()); + } else { + return false; + } + } // -------------------------------------------------------------------------------------------- @@ -272,11 +306,9 @@ public boolean equals(Object obj) { if (obj instanceof KryoSerializer) { KryoSerializer other = (KryoSerializer) obj; - // we cannot include the Serializers here because they don't implement the equals method return other.canEqual(this) && type == other.type && - registeredTypes.equals(other.registeredTypes) && - registeredTypesWithSerializerClasses.equals(other.registeredTypesWithSerializerClasses) && + kryoRegistrations.equals(other.kryoRegistrations) && defaultSerializerClasses.equals(other.defaultSerializerClasses); } else { return false; @@ -334,7 +366,7 @@ private void checkKryoInitialized() { // This is due to a know issue with Kryo's JavaSerializer. See FLINK-6025 for details. kryo.addDefaultSerializer(Throwable.class, new JavaSerializer()); - // Add default serializers first, so that they type registrations without a serializer + // Add default serializers first, so that the type registrations without a serializer // are registered with a default serializer for (Map.Entry, ExecutionConfig.SerializableSerializer> entry: defaultSerializers.entrySet()) { kryo.addDefaultSerializer(entry.getKey(), entry.getValue().getSerializer()); @@ -344,59 +376,152 @@ private void checkKryoInitialized() { kryo.addDefaultSerializer(entry.getKey(), entry.getValue()); } - // register the type of our class - kryo.register(type); + KryoUtils.applyRegistrations(this.kryo, kryoRegistrations.values()); - // register given types. we do this first so that any registration of a - // more specific serializer overrides this - for (Class type : registeredTypes) { - kryo.register(type); - } + kryo.setRegistrationRequired(false); + kryo.setClassLoader(Thread.currentThread().getContextClassLoader()); + } + } - // register given serializer classes - for (Map.Entry, Class>> e : registeredTypesWithSerializerClasses.entrySet()) { - Class typeClass = e.getKey(); - Class> serializerClass = e.getValue(); + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- - Serializer serializer = - ReflectionSerializerFactory.makeSerializer(kryo, serializerClass, typeClass); - kryo.register(typeClass, serializer); - } + @Override + public KryoSerializerConfigSnapshot snapshotConfiguration() { + return new KryoSerializerConfigSnapshot<>(type, kryoRegistrations); + } - // register given serializers - for (Map.Entry, ExecutionConfig.SerializableSerializer> e : registeredTypesWithSerializers.entrySet()) { - kryo.register(e.getKey(), e.getValue().getSerializer()); + @SuppressWarnings("unchecked") + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof KryoSerializerConfigSnapshot) { + final KryoSerializerConfigSnapshot config = (KryoSerializerConfigSnapshot) configSnapshot; + + if (type.equals(config.getTypeClass())) { + LinkedHashMap reconfiguredRegistrations = config.getKryoRegistrations(); + + // reconfigure by assuring that classes which were previously registered are registered + // again in the exact same order; new class registrations will be appended. + // this also overwrites any dummy placeholders that the restored old configuration has + reconfiguredRegistrations.putAll(kryoRegistrations); + + // check if there is still any dummy placeholders even after reconfiguration; + // if so, then this new Kryo serializer cannot read old data and is therefore incompatible + for (Map.Entry reconfiguredRegistrationEntry : reconfiguredRegistrations.entrySet()) { + if (reconfiguredRegistrationEntry.getValue().isDummy()) { + LOG.warn("The Kryo registration for a previously registered class {} does not have a " + + "proper serializer, because its previous serializer cannot be loaded or is no " + + "longer valid but a new serializer is not available", reconfiguredRegistrationEntry.getKey()); + + return MigrationStrategy.migrate(); + } + } + + // there's actually no way to tell if new Kryo serializers are compatible with + // the previous ones they overwrite; we can only signal compatibly and hope for the best + this.kryoRegistrations = reconfiguredRegistrations; + return MigrationStrategy.noMigration(); } - // this is needed for Avro but can not be added on demand. - kryo.register(GenericData.Array.class, new SpecificInstanceCollectionSerializerForArrayList()); + } - kryo.setRegistrationRequired(false); - kryo.setClassLoader(Thread.currentThread().getContextClassLoader()); + return MigrationStrategy.migrate(); + } + + public static final class KryoSerializerConfigSnapshot extends KryoRegistrationSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public KryoSerializerConfigSnapshot() {} + + public KryoSerializerConfigSnapshot( + Class typeClass, + LinkedHashMap kryoRegistrations) { + + super(typeClass, kryoRegistrations); + } + + @Override + public int getVersion() { + return VERSION; } } // -------------------------------------------------------------------------------------------- - // For testing + // Utilities // -------------------------------------------------------------------------------------------- - - public Kryo getKryo() { - checkKryoInitialized(); - return this.kryo; + + /** + * Utility method that takes lists of registered types and their serializers, and resolve + * them into a single list such that the result will resemble the final registration + * result in Kryo. + */ + private static LinkedHashMap buildKryoRegistrations( + Class serializedType, + LinkedHashSet> registeredTypes, + LinkedHashMap, Class>> registeredTypesWithSerializerClasses, + LinkedHashMap, ExecutionConfig.SerializableSerializer> registeredTypesWithSerializers) { + + final LinkedHashMap kryoRegistrations = new LinkedHashMap<>(); + + kryoRegistrations.put(serializedType.getName(), new KryoRegistration(serializedType)); + + for (Class registeredType : checkNotNull(registeredTypes)) { + kryoRegistrations.put(registeredType.getName(), new KryoRegistration(registeredType)); + } + + for (Map.Entry, Class>> registeredTypeWithSerializerClassEntry : + checkNotNull(registeredTypesWithSerializerClasses).entrySet()) { + + kryoRegistrations.put( + registeredTypeWithSerializerClassEntry.getKey().getName(), + new KryoRegistration( + registeredTypeWithSerializerClassEntry.getKey(), + registeredTypeWithSerializerClassEntry.getValue())); + } + + for (Map.Entry, ExecutionConfig.SerializableSerializer> registeredTypeWithSerializerEntry : + checkNotNull(registeredTypesWithSerializers).entrySet()) { + + kryoRegistrations.put( + registeredTypeWithSerializerEntry.getKey().getName(), + new KryoRegistration( + registeredTypeWithSerializerEntry.getKey(), + registeredTypeWithSerializerEntry.getValue())); + } + + kryoRegistrations.put( + GenericData.Array.class.getName(), + new KryoRegistration( + GenericData.Array.class, + new ExecutionConfig.SerializableSerializer<>(new Serializers.SpecificInstanceCollectionSerializerForArrayList()))); + + return kryoRegistrations; } - @Override - public boolean canRestoreFrom(TypeSerializer other) { - if (other instanceof KryoSerializer) { - KryoSerializer otherKryo = (KryoSerializer) other; + // -------------------------------------------------------------------------------------------- - // we cannot include the Serializers here because they don't implement the equals method - return other.canEqual(this) && - type == otherKryo.type && - (registeredTypes.equals(otherKryo.registeredTypes) || otherKryo.registeredTypes.isEmpty()) && - (registeredTypesWithSerializerClasses.equals(otherKryo.registeredTypesWithSerializerClasses) || otherKryo.registeredTypesWithSerializerClasses.isEmpty()) && - (defaultSerializerClasses.equals(otherKryo.defaultSerializerClasses) || otherKryo.defaultSerializerClasses.isEmpty()); - } else { - return false; + private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + + // kryoRegistrations may be null if this Kryo serializer is deserialized from an old version + if (kryoRegistrations == null) { + this.kryoRegistrations = buildKryoRegistrations( + type, + registeredTypes, + registeredTypesWithSerializerClasses, + registeredTypesWithSerializers); } } + + // -------------------------------------------------------------------------------------------- + // For testing + // -------------------------------------------------------------------------------------------- + + @VisibleForTesting + public Kryo getKryo() { + checkKryoInitialized(); + return this.kryo; + } } 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 91c6145d45d61..398a53fef4c8b 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 @@ -20,6 +20,7 @@ import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -31,6 +32,9 @@ import java.io.IOException; import java.util.Arrays; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.TestLogger; import org.junit.Assert; @@ -85,6 +89,38 @@ public void testInstantiate() { fail("Exception in test: " + e.getMessage()); } } + + @Test + public void testConfigSnapshotInstantiation() { + TypeSerializerConfigSnapshot configSnapshot = getSerializer().snapshotConfiguration(); + + InstantiationUtil.instantiate(configSnapshot.getClass()); + } + + @Test + public void testSnapshotConfigurationAndReconfigure() throws Exception { + final TypeSerializerConfigSnapshot configSnapshot = getSerializer().snapshotConfiguration(); + + byte[] serializedConfig; + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + TypeSerializerUtil.writeSerializerConfigSnapshot( + new DataOutputViewStreamWrapper(out), configSnapshot); + serializedConfig = out.toByteArray(); + } + + TypeSerializerConfigSnapshot restoredConfig; + try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { + restoredConfig = TypeSerializerUtil.readSerializerConfigSnapshot( + new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); + } + + MigrationStrategy strategy = getSerializer().getMigrationStrategyFor(restoredConfig); + assertFalse(strategy.requireMigration()); + + // also verify that the serializer's reconfigure implementation detects incompatibility + strategy = getSerializer().getMigrationStrategyFor(new TestIncompatibleSerializerConfigSnapshot()); + assertTrue(strategy.requireMigration()); + } @Test public void testGetLength() { @@ -477,4 +513,21 @@ public void skipBytesToRead(int numBytes) throws IOException { } } } + + public static final class TestIncompatibleSerializerConfigSnapshot extends TypeSerializerConfigSnapshot { + @Override + public int getVersion() { + return 0; + } + + @Override + public boolean equals(Object obj) { + return obj instanceof TestIncompatibleSerializerConfigSnapshot; + } + + @Override + public int hashCode() { + return getClass().hashCode(); + } + } } diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotTest.java new file mode 100644 index 0000000000000..6ace128185232 --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotTest.java @@ -0,0 +1,197 @@ +/* + * 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.memory.DataInputView; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.net.URL; +import java.net.URLClassLoader; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Unit tests related to {@link TypeSerializerConfigSnapshot}. + */ +public class TypeSerializerConfigSnapshotTest { + + /** + * Verifies that reading and writing configuration snapshots work correctly. + */ + @Test + public void testSerializeConfigurationSnapshots() throws Exception { + TestConfigSnapshot configSnapshot1 = new TestConfigSnapshot(1, "foo"); + TestConfigSnapshot configSnapshot2 = new TestConfigSnapshot(2, "bar"); + + byte[] serializedConfig; + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + TypeSerializerUtil.writeSerializerConfigSnapshots( + new DataOutputViewStreamWrapper(out), + configSnapshot1, + configSnapshot2); + + serializedConfig = out.toByteArray(); + } + + TypeSerializerConfigSnapshot[] restoredConfigs; + try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { + restoredConfigs = TypeSerializerUtil.readSerializerConfigSnapshots( + new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); + } + + assertEquals(2, restoredConfigs.length); + assertEquals(configSnapshot1, restoredConfigs[0]); + assertEquals(configSnapshot2, restoredConfigs[1]); + } + + /** + * Verifies that deserializing config snapshots fail if the config class could not be found. + */ + @Test + public void testFailsWhenConfigurationSnapshotClassNotFound() throws Exception { + byte[] serializedConfig; + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + TypeSerializerUtil.writeSerializerConfigSnapshot( + new DataOutputViewStreamWrapper(out), new TestConfigSnapshot(123, "foobar")); + serializedConfig = out.toByteArray(); + } + + try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { + // read using a dummy classloader + TypeSerializerUtil.readSerializerConfigSnapshot( + new DataInputViewStreamWrapper(in), new URLClassLoader(new URL[0], null)); + fail("Expected a ClassNotFoundException wrapped in IOException"); + } catch (IOException expected) { + // test passes + } + } + + /** + * Tests that serializing and then deserializing the special marker config + * {@link ForwardCompatibleSerializationFormatConfig#INSTANCE} always + * restores the singleton instance. + */ + @Test + public void testSerializeForwardCompatibleMarkerConfig() throws Exception { + byte[] serializedConfig; + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + TypeSerializerUtil.writeSerializerConfigSnapshot( + new DataOutputViewStreamWrapper(out), ForwardCompatibleSerializationFormatConfig.INSTANCE); + serializedConfig = out.toByteArray(); + } + + TypeSerializerConfigSnapshot restoredConfig; + try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { + restoredConfig = TypeSerializerUtil.readSerializerConfigSnapshot( + new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); + } + + // reference equality to the singleton instance + assertTrue(restoredConfig == ForwardCompatibleSerializationFormatConfig.INSTANCE); + } + + /** + * Verifies that the actual reconfigure method is never invoked if the + * provided configuration snapshot is the special singleton marker config + * {@link ForwardCompatibleSerializationFormatConfig#INSTANCE}. + */ + @Test + public void testMigrationStrategyWithForwardCompatibleMarkerConfig() { + TypeSerializer mockSerializer = spy(TypeSerializer.class); + + mockSerializer.getMigrationStrategyFor(ForwardCompatibleSerializationFormatConfig.INSTANCE); + verify(mockSerializer, never()).getMigrationStrategy(any(TypeSerializerConfigSnapshot.class)); + + // make sure that is actually is called if its not the special marker + TypeSerializerConfigSnapshot nonForwardCompatibleConfig = new TestConfigSnapshot(123, "foobar"); + mockSerializer.getMigrationStrategyFor(nonForwardCompatibleConfig); + verify(mockSerializer, times(1)).getMigrationStrategy(nonForwardCompatibleConfig); + } + + public static class TestConfigSnapshot extends TypeSerializerConfigSnapshot { + + static final int VERSION = 1; + + private int val; + private String msg; + + public TestConfigSnapshot() {} + + public TestConfigSnapshot(int val, String msg) { + this.val = val; + this.msg = msg; + } + + @Override + public void write(DataOutputView out) throws IOException { + super.write(out); + out.writeInt(val); + out.writeUTF(msg); + } + + @Override + public void read(DataInputView in) throws IOException { + super.read(in); + val = in.readInt(); + msg = in.readUTF(); + } + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + + if (obj == null) { + return false; + } + + if (obj instanceof TestConfigSnapshot) { + return val == ((TestConfigSnapshot) obj).val && msg.equals(((TestConfigSnapshot) obj).msg); + } else { + return false; + } + } + + @Override + public int hashCode() { + return 31 * val + msg.hashCode(); + } + } +} 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 5e2e733a8c2e1..4236dbd8c097a 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerTest.java @@ -18,11 +18,25 @@ package org.apache.flink.api.common.typeutils.base; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.SerializerTestInstance; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerUtil; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.TestLogger; import org.junit.Test; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.util.Arrays; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + public class EnumSerializerTest extends TestLogger { @Test @@ -41,6 +55,132 @@ public void testEmptyEnum() { new EnumSerializer<>(EmptyEnum.class); } + @Test + public void testReconfiguration() { + // mock the previous ordering of enum constants to be BAR, PAULA, NATHANIEL + PublicEnum[] mockPreviousOrder = {PublicEnum.BAR, PublicEnum.PAULA, PublicEnum.NATHANIEL}; + + // now, the actual order of FOO, BAR, PETER, NATHANIEL, EMMA, PAULA will be the "new wrong order" + EnumSerializer serializer = new EnumSerializer<>(PublicEnum.class); + + // verify that the serializer is first using the "wrong order" (i.e., the initial new configuration) + assertEquals(PublicEnum.FOO.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.FOO).intValue()); + assertEquals(PublicEnum.BAR.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.BAR).intValue()); + assertEquals(PublicEnum.PETER.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.PETER).intValue()); + assertEquals(PublicEnum.NATHANIEL.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.NATHANIEL).intValue()); + assertEquals(PublicEnum.EMMA.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.EMMA).intValue()); + assertEquals(PublicEnum.PAULA.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.PAULA).intValue()); + + // reconfigure and verify compatibility + MigrationStrategy strategy = serializer.getMigrationStrategyFor( + new EnumSerializer.EnumSerializerConfigSnapshot<>(PublicEnum.class, mockPreviousOrder)); + assertFalse(strategy.requireMigration()); + + // after reconfiguration, the order should be first the original BAR, PAULA, NATHANIEL, + // followed by the "new enum constants" FOO, PETER, EMMA + PublicEnum[] expectedOrder = {PublicEnum.BAR, PublicEnum.PAULA, PublicEnum.NATHANIEL, PublicEnum.FOO, PublicEnum.PETER, PublicEnum.EMMA}; + + int i = 0; + for (PublicEnum constant : expectedOrder) { + assertEquals(i, serializer.getValueToOrdinal().get(constant).intValue()); + i++; + } + + assertTrue(Arrays.equals(expectedOrder, serializer.getValues())); + } + + @Test + public void testConfigurationSnapshotSerialization() throws Exception { + EnumSerializer serializer = new EnumSerializer<>(PublicEnum.class); + + byte[] serializedConfig; + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + TypeSerializerUtil.writeSerializerConfigSnapshot( + new DataOutputViewStreamWrapper(out), serializer.snapshotConfiguration()); + serializedConfig = out.toByteArray(); + } + + TypeSerializerConfigSnapshot restoredConfig; + try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { + restoredConfig = TypeSerializerUtil.readSerializerConfigSnapshot( + new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); + } + + MigrationStrategy strategy = serializer.getMigrationStrategyFor(restoredConfig); + assertFalse(strategy.requireMigration()); + + assertEquals(PublicEnum.FOO.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.FOO).intValue()); + assertEquals(PublicEnum.BAR.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.BAR).intValue()); + assertEquals(PublicEnum.PETER.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.PETER).intValue()); + assertEquals(PublicEnum.NATHANIEL.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.NATHANIEL).intValue()); + assertEquals(PublicEnum.EMMA.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.EMMA).intValue()); + assertEquals(PublicEnum.PAULA.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.PAULA).intValue()); + assertTrue(Arrays.equals(PublicEnum.values(), serializer.getValues())); + } + + @Test + public void testSerializeEnumSerializer() throws Exception { + EnumSerializer serializer = new EnumSerializer<>(PublicEnum.class); + + // verify original transient parameters + assertEquals(PublicEnum.FOO.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.FOO).intValue()); + assertEquals(PublicEnum.BAR.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.BAR).intValue()); + assertEquals(PublicEnum.PETER.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.PETER).intValue()); + assertEquals(PublicEnum.NATHANIEL.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.NATHANIEL).intValue()); + assertEquals(PublicEnum.EMMA.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.EMMA).intValue()); + assertEquals(PublicEnum.PAULA.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.PAULA).intValue()); + assertTrue(Arrays.equals(PublicEnum.values(), serializer.getValues())); + + byte[] serializedSerializer = InstantiationUtil.serializeObject(serializer); + + // deserialize and re-verify transient parameters + serializer = InstantiationUtil.deserializeObject(serializedSerializer, Thread.currentThread().getContextClassLoader()); + assertEquals(PublicEnum.FOO.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.FOO).intValue()); + assertEquals(PublicEnum.BAR.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.BAR).intValue()); + assertEquals(PublicEnum.PETER.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.PETER).intValue()); + assertEquals(PublicEnum.NATHANIEL.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.NATHANIEL).intValue()); + assertEquals(PublicEnum.EMMA.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.EMMA).intValue()); + assertEquals(PublicEnum.PAULA.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.PAULA).intValue()); + assertTrue(Arrays.equals(PublicEnum.values(), serializer.getValues())); + } + + @Test + public void testSerializeReconfiguredEnumSerializer() throws Exception { + // mock the previous ordering of enum constants to be BAR, PAULA, NATHANIEL + PublicEnum[] mockPreviousOrder = {PublicEnum.BAR, PublicEnum.PAULA, PublicEnum.NATHANIEL}; + + // now, the actual order of FOO, BAR, PETER, NATHANIEL, EMMA, PAULA will be the "new wrong order" + EnumSerializer serializer = new EnumSerializer<>(PublicEnum.class); + + // verify that the serializer is first using the "wrong order" (i.e., the initial new configuration) + assertEquals(PublicEnum.FOO.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.FOO).intValue()); + assertEquals(PublicEnum.BAR.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.BAR).intValue()); + assertEquals(PublicEnum.PETER.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.PETER).intValue()); + assertEquals(PublicEnum.NATHANIEL.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.NATHANIEL).intValue()); + assertEquals(PublicEnum.EMMA.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.EMMA).intValue()); + assertEquals(PublicEnum.PAULA.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.PAULA).intValue()); + + // reconfigure and verify compatibility + MigrationStrategy strategy = serializer.getMigrationStrategyFor( + new EnumSerializer.EnumSerializerConfigSnapshot<>(PublicEnum.class, mockPreviousOrder)); + assertFalse(strategy.requireMigration()); + + // serialize and deserialize again the serializer + byte[] serializedSerializer = InstantiationUtil.serializeObject(serializer); + serializer = InstantiationUtil.deserializeObject(serializedSerializer, Thread.currentThread().getContextClassLoader()); + + // verify that after the serializer was read, the reconfigured constant ordering is untouched + PublicEnum[] expectedOrder = {PublicEnum.BAR, PublicEnum.PAULA, PublicEnum.NATHANIEL, PublicEnum.FOO, PublicEnum.PETER, PublicEnum.EMMA}; + + int i = 0; + for (PublicEnum constant : expectedOrder) { + assertEquals(i, serializer.getValueToOrdinal().get(constant).intValue()); + i++; + } + + assertTrue(Arrays.equals(expectedOrder, serializer.getValues())); + } + @SafeVarargs public final > void testEnumSerializer(T... data) { @SuppressWarnings("unchecked") diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/array/CharPrimitiveArraySerializerTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/array/CharPrimitiveArraySerializerTest.java index 9fda3d096ec4a..3301aa2071fc1 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/array/CharPrimitiveArraySerializerTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/array/CharPrimitiveArraySerializerTest.java @@ -20,8 +20,6 @@ import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.base.array.CharPrimitiveArraySerializer; -import org.apache.flink.api.common.typeutils.base.array.LongPrimitiveArraySerializer; /** * A test for the {@link LongPrimitiveArraySerializer}. 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 4712ed1ac04dd..fbb19bcf7f31b 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 @@ -18,29 +18,44 @@ package org.apache.flink.api.java.typeutils.runtime; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.lang.reflect.Field; import java.util.ArrayList; import java.util.Date; +import java.util.HashMap; +import java.util.LinkedHashMap; import java.util.List; import java.util.Objects; import java.util.Random; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.ForwardCompatibleSerializationFormatConfig; import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.CompositeType.FlatFieldDescriptor; 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.TypeSerializerUtil; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.PojoTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.junit.Assert; import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + /** * A test for the {@link PojoSerializer}. */ @@ -191,6 +206,20 @@ public boolean equals(Object other) { return true; } } + + public static class SubTestUserClassA extends TestUserClass { + public int subDumm1; + public String subDumm2; + + public SubTestUserClassA() {} + } + + public static class SubTestUserClassB extends TestUserClass { + public Double subDumm1; + public float subDumm2; + + public SubTestUserClassB() {} + } /** * This tests if the hashes returned by the pojo and tuple comparators are the same @@ -240,4 +269,237 @@ public void testTuplePojoTestEquality() { Assert.assertTrue("The hashing for tuples and pojos must be the same, so that they are mixable. Also for those with multiple key fields", multiPojoHash == multiTupleHash); } -} + + // -------------------------------------------------------------------------------------------- + // Configuration snapshotting & reconfiguring tests + // -------------------------------------------------------------------------------------------- + + /** + * Verifies that reconfiguring with a config snapshot of a preceding POJO serializer + * with different POJO type will result in INCOMPATIBLE. + */ + @Test + public void testReconfigureWithDifferentPojoType() throws Exception { + PojoSerializer pojoSerializer1 = (PojoSerializer) + TypeExtractor.getForClass(SubTestUserClassB.class).createSerializer(new ExecutionConfig()); + + // snapshot configuration and serialize to bytes + TypeSerializerConfigSnapshot pojoSerializerConfigSnapshot = pojoSerializer1.snapshotConfiguration(); + byte[] serializedConfig; + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + TypeSerializerUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), pojoSerializerConfigSnapshot); + serializedConfig = out.toByteArray(); + } + + PojoSerializer pojoSerializer2 = (PojoSerializer) + TypeExtractor.getForClass(SubTestUserClassA.class).createSerializer(new ExecutionConfig()); + + // read configuration again from bytes + try(ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { + pojoSerializerConfigSnapshot = TypeSerializerUtil.readSerializerConfigSnapshot( + new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); + } + + MigrationStrategy strategy = pojoSerializer2.getMigrationStrategy(pojoSerializerConfigSnapshot); + assertTrue(strategy.requireMigration()); + } + + /** + * Tests that reconfiguration correctly reorders subclass registrations to their previous order. + */ + @Test + public void testReconfigureDifferentSubclassRegistrationOrder() throws Exception { + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.registerPojoType(SubTestUserClassA.class); + executionConfig.registerPojoType(SubTestUserClassB.class); + + PojoSerializer pojoSerializer = (PojoSerializer) type.createSerializer(executionConfig); + + // get original registration ids + int subClassATag = pojoSerializer.getRegisteredClasses().get(SubTestUserClassA.class); + int subClassBTag = pojoSerializer.getRegisteredClasses().get(SubTestUserClassB.class); + + // snapshot configuration and serialize to bytes + TypeSerializerConfigSnapshot pojoSerializerConfigSnapshot = pojoSerializer.snapshotConfiguration(); + byte[] serializedConfig; + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + TypeSerializerUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), pojoSerializerConfigSnapshot); + serializedConfig = out.toByteArray(); + } + + // use new config and instantiate new PojoSerializer + executionConfig = new ExecutionConfig(); + executionConfig.registerPojoType(SubTestUserClassB.class); // test with B registered before A + executionConfig.registerPojoType(SubTestUserClassA.class); + + pojoSerializer = (PojoSerializer) type.createSerializer(executionConfig); + + // read configuration from bytes + try(ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { + pojoSerializerConfigSnapshot = TypeSerializerUtil.readSerializerConfigSnapshot( + new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); + } + + MigrationStrategy result = pojoSerializer.getMigrationStrategy(pojoSerializerConfigSnapshot); + assertTrue(!result.requireMigration()); + + // reconfigure - check reconfiguration result and that registration ids remains the same + //assertEquals(ReconfigureResult.COMPATIBLE, pojoSerializer.reconfigure(pojoSerializerConfigSnapshot)); + assertEquals(subClassATag, pojoSerializer.getRegisteredClasses().get(SubTestUserClassA.class).intValue()); + assertEquals(subClassBTag, pojoSerializer.getRegisteredClasses().get(SubTestUserClassB.class).intValue()); + } + + /** + * Tests that reconfiguration repopulates previously cached subclass serializers. + */ + @Test + public void testReconfigureRepopulateNonregisteredSubclassSerializerCache() throws Exception { + // don't register any subclasses + PojoSerializer pojoSerializer = (PojoSerializer) type.createSerializer(new ExecutionConfig()); + + // create cached serializers for SubTestUserClassA and SubTestUserClassB + pojoSerializer.getSubclassSerializer(SubTestUserClassA.class); + pojoSerializer.getSubclassSerializer(SubTestUserClassB.class); + + assertEquals(2, pojoSerializer.getSubclassSerializerCache().size()); + assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassA.class)); + assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassB.class)); + + // snapshot configuration and serialize to bytes + TypeSerializerConfigSnapshot pojoSerializerConfigSnapshot = pojoSerializer.snapshotConfiguration(); + byte[] serializedConfig; + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + TypeSerializerUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), pojoSerializerConfigSnapshot); + serializedConfig = out.toByteArray(); + } + + // instantiate new PojoSerializer + + pojoSerializer = (PojoSerializer) type.createSerializer(new ExecutionConfig()); + + // read configuration from bytes + try(ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { + pojoSerializerConfigSnapshot = TypeSerializerUtil.readSerializerConfigSnapshot( + new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); + } + + // reconfigure - check reconfiguration result and that subclass serializer cache is repopulated + MigrationStrategy strategy = pojoSerializer.getMigrationStrategyFor(pojoSerializerConfigSnapshot); + assertFalse(strategy.requireMigration()); + assertEquals(2, pojoSerializer.getSubclassSerializerCache().size()); + assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassA.class)); + assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassB.class)); + } + + /** + * Tests that: + * - Previous Pojo serializer did not have registrations, and created cached serializers for subclasses + * - On restore, it had those subclasses registered + * + * In this case, after reconfiguration, the cache should be repopulated, and registrations should + * also exist for the subclasses. + * + * Note: the cache still needs to be repopulated because previous data of those subclasses were + * written with the cached serializers. In this case, the repopulated cache has reconfigured serializers + * for the subclasses so that previous written data can be read, but the registered serializers + * for the subclasses do not necessarily need to be reconfigured since they will only be used to + * write new data. + */ + @Test + public void testReconfigureWithPreviouslyNonregisteredSubclasses() throws Exception { + // don't register any subclasses at first + PojoSerializer pojoSerializer = (PojoSerializer) type.createSerializer(new ExecutionConfig()); + + // create cached serializers for SubTestUserClassA and SubTestUserClassB + pojoSerializer.getSubclassSerializer(SubTestUserClassA.class); + pojoSerializer.getSubclassSerializer(SubTestUserClassB.class); + + // make sure serializers are in cache + assertEquals(2, pojoSerializer.getSubclassSerializerCache().size()); + assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassA.class)); + assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassB.class)); + + // make sure that registrations are empty + assertTrue(pojoSerializer.getRegisteredClasses().isEmpty()); + assertEquals(0, pojoSerializer.getRegisteredSerializers().length); + + // snapshot configuration and serialize to bytes + TypeSerializerConfigSnapshot pojoSerializerConfigSnapshot = pojoSerializer.snapshotConfiguration(); + byte[] serializedConfig; + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + TypeSerializerUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), pojoSerializerConfigSnapshot); + serializedConfig = out.toByteArray(); + } + + // instantiate new PojoSerializer, with new execution config that has the subclass registrations + ExecutionConfig newExecutionConfig = new ExecutionConfig(); + newExecutionConfig.registerPojoType(SubTestUserClassA.class); + newExecutionConfig.registerPojoType(SubTestUserClassB.class); + pojoSerializer = (PojoSerializer) type.createSerializer(newExecutionConfig); + + // read configuration from bytes + try(ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { + pojoSerializerConfigSnapshot = TypeSerializerUtil.readSerializerConfigSnapshot( + new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); + } + + // reconfigure - check reconfiguration result and that + // 1) subclass serializer cache is repopulated + // 2) registrations also contain the now registered subclasses + MigrationStrategy strategy = pojoSerializer.getMigrationStrategyFor(pojoSerializerConfigSnapshot); + assertFalse(strategy.requireMigration()); + assertEquals(2, pojoSerializer.getSubclassSerializerCache().size()); + assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassA.class)); + assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassB.class)); + assertEquals(2, pojoSerializer.getRegisteredClasses().size()); + assertTrue(pojoSerializer.getRegisteredClasses().containsKey(SubTestUserClassA.class)); + assertTrue(pojoSerializer.getRegisteredClasses().containsKey(SubTestUserClassB.class)); + } + + /** + * Verifies that reconfiguration reorders the fields of the new Pojo serializer to remain the same. + */ + @Test + public void testReconfigureWithDifferentFieldOrder() throws Exception { + Field[] mockOriginalFieldOrder = { + TestUserClass.class.getField("dumm4"), + TestUserClass.class.getField("dumm3"), + TestUserClass.class.getField("nestedClass"), + TestUserClass.class.getField("dumm1"), + TestUserClass.class.getField("dumm2"), + TestUserClass.class.getField("dumm5"), + }; + + LinkedHashMap mockOriginalFieldToSerializerConfigSnapshot = + new LinkedHashMap<>(mockOriginalFieldOrder.length); + for (Field field : mockOriginalFieldOrder) { + // just use forward compatible marker; just the field ordering is relevant for this test + mockOriginalFieldToSerializerConfigSnapshot.put(field, ForwardCompatibleSerializationFormatConfig.INSTANCE); + } + + PojoSerializer pojoSerializer = (PojoSerializer) type.createSerializer(new ExecutionConfig()); + + assertEquals(TestUserClass.class.getField("dumm1"), pojoSerializer.getFields()[0]); + assertEquals(TestUserClass.class.getField("dumm2"), pojoSerializer.getFields()[1]); + assertEquals(TestUserClass.class.getField("dumm3"), pojoSerializer.getFields()[2]); + assertEquals(TestUserClass.class.getField("dumm4"), pojoSerializer.getFields()[3]); + assertEquals(TestUserClass.class.getField("dumm5"), pojoSerializer.getFields()[4]); + assertEquals(TestUserClass.class.getField("nestedClass"), pojoSerializer.getFields()[5]); + + PojoSerializer.PojoSerializerConfigSnapshot mockPreviousConfigSnapshot = + new PojoSerializer.PojoSerializerConfigSnapshot<>( + TestUserClass.class, + mockOriginalFieldToSerializerConfigSnapshot, // this mocks the previous field order + new LinkedHashMap, TypeSerializerConfigSnapshot>(), // empty; irrelevant for this test + new HashMap, TypeSerializerConfigSnapshot>()); // empty; irrelevant for this test + + // reconfigure - check reconfiguration result and that fields are reordered to the previous order + MigrationStrategy strategy = pojoSerializer.getMigrationStrategyFor(mockPreviousConfigSnapshot); + assertFalse(strategy.requireMigration()); + int i = 0; + for (Field field : mockOriginalFieldOrder) { + assertEquals(field, pojoSerializer.getFields()[i]); + i++; + } + } +} diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerMigrationTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerMigrationTest.java new file mode 100644 index 0000000000000..b1cea46efe0db --- /dev/null +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerMigrationTest.java @@ -0,0 +1,136 @@ +/* + * 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.java.typeutils.runtime.kryo; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.Serializer; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializerUtil; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Tests related to configuration snapshotting and reconfiguring for the {@link KryoSerializer}. + */ +public class KryoSerializerMigrationTest { + + /** + * Verifies that reconfiguration result is INCOMPATIBLE if data type has changed. + */ + @Test + public void testMigrationStrategyWithDifferentKryoType() throws Exception { + KryoSerializer kryoSerializerForA = new KryoSerializer<>(TestClassA.class, new ExecutionConfig()); + + // snapshot configuration and serialize to bytes + TypeSerializerConfigSnapshot kryoSerializerConfigSnapshot = kryoSerializerForA.snapshotConfiguration(); + byte[] serializedConfig; + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + TypeSerializerUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), kryoSerializerConfigSnapshot); + serializedConfig = out.toByteArray(); + } + + KryoSerializer kryoSerializerForB = new KryoSerializer<>(TestClassB.class, new ExecutionConfig()); + + // read configuration again from bytes + try(ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { + kryoSerializerConfigSnapshot = TypeSerializerUtil.readSerializerConfigSnapshot( + new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); + } + + MigrationStrategy strategy = kryoSerializerForB.getMigrationStrategy(kryoSerializerConfigSnapshot); + assertTrue(strategy.requireMigration()); + } + + /** + * Tests that after reconfiguration, registration ids are reconfigured to + * remain the same as the preceding KryoSerializer. + */ + @Test + public void testMigrationStrategyForDifferentRegistrationOrder() throws Exception { + + ExecutionConfig executionConfig = new ExecutionConfig(); + executionConfig.registerKryoType(TestClassA.class); + executionConfig.registerKryoType(TestClassB.class); + + KryoSerializer kryoSerializer = new KryoSerializer<>(TestClass.class, executionConfig); + + // get original registration ids + int testClassId = kryoSerializer.getKryo().getRegistration(TestClass.class).getId(); + int testClassAId = kryoSerializer.getKryo().getRegistration(TestClassA.class).getId(); + int testClassBId = kryoSerializer.getKryo().getRegistration(TestClassB.class).getId(); + + // snapshot configuration and serialize to bytes + TypeSerializerConfigSnapshot kryoSerializerConfigSnapshot = kryoSerializer.snapshotConfiguration(); + byte[] serializedConfig; + try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { + TypeSerializerUtil.writeSerializerConfigSnapshot(new DataOutputViewStreamWrapper(out), kryoSerializerConfigSnapshot); + serializedConfig = out.toByteArray(); + } + + // use new config and instantiate new KryoSerializer + executionConfig = new ExecutionConfig(); + executionConfig.registerKryoType(TestClassB.class); // test with B registered before A + executionConfig.registerKryoType(TestClassA.class); + + kryoSerializer = new KryoSerializer<>(TestClass.class, executionConfig); + + // read configuration from bytes + try(ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { + kryoSerializerConfigSnapshot = TypeSerializerUtil.readSerializerConfigSnapshot( + new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); + } + + // reconfigure - check reconfiguration result and that registration id remains the same + MigrationStrategy strategy = kryoSerializer.getMigrationStrategyFor(kryoSerializerConfigSnapshot); + assertFalse(strategy.requireMigration()); + assertEquals(testClassId, kryoSerializer.getKryo().getRegistration(TestClass.class).getId()); + assertEquals(testClassAId, kryoSerializer.getKryo().getRegistration(TestClassA.class).getId()); + assertEquals(testClassBId, kryoSerializer.getKryo().getRegistration(TestClassB.class).getId()); + } + + private static class TestClass {} + + private static class TestClassA {} + + private static class TestClassB {} + + private static class TestClassBSerializer extends Serializer { + @Override + public void write(Kryo kryo, Output output, Object o) { + throw new UnsupportedOperationException(); + } + + @Override + public Object read(Kryo kryo, Input input, Class aClass) { + throw new UnsupportedOperationException(); + } + } +} 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 3bb40ebcb2f38..6024ad5d96f08 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 @@ -26,7 +26,9 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.core.io.GenericInputSplit; import org.apache.flink.core.memory.DataInputView; @@ -386,5 +388,15 @@ public boolean canEqual(Object obj) { public int hashCode() { return Objects.hash(failOnRead, failOnWrite); } + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + throw new UnsupportedOperationException(); + } + + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + throw new UnsupportedOperationException(); + } } } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java index 846b6c3133927..3ad3f03278920 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java @@ -18,7 +18,10 @@ package org.apache.flink.cep; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -39,7 +42,8 @@ * * @param Type of the element to be serialized */ -public class NonDuplicatingTypeSerializer extends TypeSerializer { +@Internal +public final class NonDuplicatingTypeSerializer extends TypeSerializer { private static final long serialVersionUID = -7633631762221447524L; // underlying type serializer @@ -192,4 +196,14 @@ private void readObject(ObjectInputStream ois) throws IOException, ClassNotFound this.identityMap = new IdentityHashMap<>(); this.elementList = new ArrayList<>(); } + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + throw new UnsupportedOperationException("This serializer is not registered for managed state."); + } + + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + throw new UnsupportedOperationException("This serializer is not registered for managed state."); + } } 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 b8c4e65c223c2..70755e553f904 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 @@ -21,6 +21,22 @@ import com.google.common.base.Predicate; import com.google.common.collect.Iterators; import com.google.common.collect.LinkedHashMultimap; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream; +import org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream; +import org.apache.flink.cep.NonDuplicatingTypeSerializer; +import org.apache.flink.cep.nfa.compiler.NFACompiler; +import org.apache.flink.cep.pattern.conditions.IterativeCondition; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -43,20 +59,6 @@ import java.util.Stack; import java.util.regex.Matcher; import java.util.regex.Pattern; -import javax.annotation.Nullable; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream; -import org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream; -import org.apache.flink.cep.NonDuplicatingTypeSerializer; -import org.apache.flink.cep.nfa.compiler.NFACompiler; -import org.apache.flink.cep.pattern.conditions.IterativeCondition; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataInputViewStreamWrapper; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.core.memory.DataOutputViewStreamWrapper; -import org.apache.flink.streaming.api.windowing.time.Time; -import org.apache.flink.util.Preconditions; /** * Non-deterministic finite automaton implementation. @@ -859,7 +861,7 @@ private ComputationState readComputationState(ObjectInputStream ois) throws I /** * A {@link TypeSerializer} for {@link NFA} that uses Java Serialization. */ - public static class Serializer extends TypeSerializer> { + public static class Serializer extends TypeSerializerSingleton> { private static final long serialVersionUID = 1L; @@ -868,11 +870,6 @@ public boolean isImmutableType() { return false; } - @Override - public TypeSerializer> duplicate() { - return this; - } - @Override public NFA createInstance() { return null; @@ -943,19 +940,9 @@ public void copy(DataInputView source, DataOutputView target) throws IOException target.write(source, size); } - @Override - public boolean equals(Object obj) { - return obj instanceof Serializer && ((Serializer) obj).canEqual(this); - } - @Override public boolean canEqual(Object obj) { return obj instanceof Serializer; } - - @Override - public int hashCode() { - return getClass().hashCode(); - } } } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java index b6374cdc441cc..2997416ac8317 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java @@ -21,7 +21,10 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.base.CollectionSerializerConfigSnapshot; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.cep.nfa.NFA; import org.apache.flink.cep.nfa.compiler.NFACompiler; @@ -498,6 +501,38 @@ public boolean canEqual(Object obj) { public int hashCode() { return Objects.hash(factory, elementSerializer); } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + return new CollectionSerializerConfigSnapshot(elementSerializer.snapshotConfiguration()); + } + + @Override + protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof CollectionSerializerConfigSnapshot) { + MigrationStrategy strategy = elementSerializer.getMigrationStrategyFor( + ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); + + if (strategy.requireMigration()) { + if (strategy.getFallbackDeserializer() != null) { + return MigrationStrategy.migrateWithFallbackDeserializer( + new PriorityQueueSerializer<>( + strategy.getFallbackDeserializer(), + factory)); + } else { + return MigrationStrategy.migrate(); + } + } else { + return MigrationStrategy.noMigration(); + } + } else { + return MigrationStrategy.migrate(); + } + } } private interface PriorityQueueFactory extends Serializable { diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializer.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializer.java index b86fe87ed427f..59841226314aa 100644 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializer.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/IntValueArraySerializer.java @@ -19,6 +19,7 @@ package org.apache.flink.graph.types.valuearray; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.api.common.typeutils.base.array.IntPrimitiveArraySerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -82,4 +83,10 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof IntValueArraySerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(IntPrimitiveArraySerializer.class.getCanonicalName()); + } } diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializer.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializer.java index 95219b6824b98..e95a1a721b78a 100644 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializer.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/LongValueArraySerializer.java @@ -19,6 +19,7 @@ package org.apache.flink.graph.types.valuearray; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.api.common.typeutils.base.array.LongPrimitiveArraySerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -82,4 +83,10 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof LongValueArraySerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(LongPrimitiveArraySerializer.class.getCanonicalName()); + } } diff --git a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializer.java b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializer.java index 0e875e35d73a6..6dbe0e5e1d72b 100644 --- a/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializer.java +++ b/flink-libraries/flink-gelly/src/main/java/org/apache/flink/graph/types/valuearray/StringValueArraySerializer.java @@ -19,6 +19,7 @@ package org.apache.flink.graph.types.valuearray; import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; +import org.apache.flink.api.common.typeutils.base.array.StringArraySerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -82,4 +83,10 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof StringValueArraySerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(StringArraySerializer.class.getCanonicalName()); + } } 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 1f56a985966c6..dd319d34ec855 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 @@ -18,7 +18,7 @@ package org.apache.flink.table.runtime.types -import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.{CompositeTypeSerializerConfigSnapshot, MigrationStrategy, TypeSerializer, TypeSerializerConfigSnapshot} import org.apache.flink.core.memory.{DataInputView, DataOutputView} import org.apache.flink.types.Row @@ -75,4 +75,51 @@ class CRowSerializer(val rowSerializer: TypeSerializer[Row]) extends TypeSeriali } override def hashCode: Int = rowSerializer.hashCode() * 13 + + override def snapshotConfiguration(): TypeSerializerConfigSnapshot = { + new CRowSerializer.CRowSerializerConfigSnapshot( + rowSerializer.snapshotConfiguration()) + } + + override protected def getMigrationStrategy( + configSnapshot: TypeSerializerConfigSnapshot): MigrationStrategy[CRow] = { + + configSnapshot match { + case crowSerializerConfigSnapshot: CRowSerializer.CRowSerializerConfigSnapshot => + val strategy = rowSerializer.getMigrationStrategyFor( + crowSerializerConfigSnapshot.getSingleNestedSerializerConfigSnapshot) + + if (strategy.requireMigration()) { + if (strategy.getFallbackDeserializer != null) { + MigrationStrategy.migrateWithFallbackDeserializer( + new CRowSerializer(strategy.getFallbackDeserializer) + ) + } else { + MigrationStrategy.migrate() + } + } else { + MigrationStrategy.noMigration() + } + + case _ => MigrationStrategy.migrate() + } + } +} + +object CRowSerializer { + + class CRowSerializerConfigSnapshot( + private var rowSerializerConfigSnapshot: TypeSerializerConfigSnapshot) + extends CompositeTypeSerializerConfigSnapshot(rowSerializerConfigSnapshot) { + + /** This empty nullary constructor is required for deserializing the configuration. */ + def this() = this(null) + + override def getVersion: Int = CRowSerializerConfigSnapshot.VERSION + } + + object CRowSerializerConfigSnapshot { + val VERSION = 1 + } + } diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java index c6813b6863ebe..570df50608780 100644 --- a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java +++ b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java @@ -18,7 +18,9 @@ package org.apache.flink.migration; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -102,6 +104,18 @@ public void copy(DataInputView source, DataOutputView target) throws IOException "This is just a proxy used during migration until the real type serializer is provided by the user."); } + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + throw new UnsupportedOperationException( + "This is just a proxy used during migration until the real type serializer is provided by the user."); + } + + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + throw new UnsupportedOperationException( + "This is just a proxy used during migration until the real type serializer is provided by the user."); + } + @Override public boolean equals(Object obj) { return obj instanceof MigrationNamespaceSerializerProxy; 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 0badb415f97b9..c0ec116ce6beb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ArrayListSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ArrayListSerializer.java @@ -17,7 +17,10 @@ */ package org.apache.flink.runtime.state; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.base.CollectionSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -132,4 +135,34 @@ public boolean canEqual(Object obj) { public int hashCode() { return elementSerializer.hashCode(); } -} \ No newline at end of file + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + return new CollectionSerializerConfigSnapshot(elementSerializer.snapshotConfiguration()); + } + + @Override + protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof CollectionSerializerConfigSnapshot) { + MigrationStrategy strategy = elementSerializer.getMigrationStrategyFor( + ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); + + if (strategy.requireMigration()) { + if (strategy.getFallbackDeserializer() != null) { + return MigrationStrategy.migrateWithFallbackDeserializer( + new ArrayListSerializer<>(strategy.getFallbackDeserializer())); + } else { + return MigrationStrategy.migrate(); + } + } else { + return MigrationStrategy.noMigration(); + } + } else { + return MigrationStrategy.migrate(); + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/HashMapSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/HashMapSerializer.java index 61cc58ca32886..0377ddb8f869b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/HashMapSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/HashMapSerializer.java @@ -18,7 +18,11 @@ package org.apache.flink.runtime.state; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.base.MapSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.util.Preconditions; @@ -38,7 +42,8 @@ * @param The type of the keys in the map. * @param The type of the values in the map. */ -public class HashMapSerializer extends TypeSerializer> { +@Internal +public final class HashMapSerializer extends TypeSerializer> { private static final long serialVersionUID = -6885593032367050078L; @@ -190,4 +195,41 @@ public boolean canEqual(Object obj) { public int hashCode() { return keySerializer.hashCode() * 31 + valueSerializer.hashCode(); } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + return new MapSerializerConfigSnapshot( + keySerializer.snapshotConfiguration(), + valueSerializer.snapshotConfiguration()); + } + + @Override + protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof MapSerializerConfigSnapshot) { + TypeSerializerConfigSnapshot[] keyValueSerializerConfigSnapshots = + ((MapSerializerConfigSnapshot) configSnapshot).getNestedSerializerConfigSnapshots(); + + MigrationStrategy keyStrategy = keySerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[0]); + MigrationStrategy valueStrategy = valueSerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[1]); + + if (keyStrategy.requireMigration() || valueStrategy.requireMigration()) { + if (keyStrategy.getFallbackDeserializer() != null && valueStrategy.getFallbackDeserializer() != null) { + return MigrationStrategy.migrateWithFallbackDeserializer( + new HashMapSerializer<>( + keyStrategy.getFallbackDeserializer(), + valueStrategy.getFallbackDeserializer())); + } else { + return MigrationStrategy.migrate(); + } + } else { + return MigrationStrategy.noMigration(); + } + } else { + return MigrationStrategy.migrate(); + } + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/JavaSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/JavaSerializer.java index 512baf6cf10d0..d49b1d22112d6 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/JavaSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/JavaSerializer.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.state; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream; import org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream; import org.apache.flink.core.memory.DataInputView; @@ -31,7 +31,7 @@ @SuppressWarnings("serial") @Internal -final class JavaSerializer extends TypeSerializer { +final class JavaSerializer extends TypeSerializerSingleton { private static final long serialVersionUID = 5067491650263321234L; @@ -40,11 +40,6 @@ public boolean isImmutableType() { return false; } - @Override - public TypeSerializer duplicate() { - return this; - } - @Override public T createInstance() { return null; @@ -97,18 +92,8 @@ public void copy(DataInputView source, DataOutputView target) throws IOException target.write(source, size); } - @Override - public boolean equals(Object obj) { - return obj instanceof JavaSerializer; - } - @Override public boolean canEqual(Object obj) { return obj instanceof JavaSerializer; } - - @Override - public int hashCode() { - return getClass().hashCode(); - } } 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 e872526b5f593..ee80878f0ac07 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListSerializer.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListSerializer.java @@ -21,7 +21,9 @@ import java.io.IOException; import java.util.Arrays; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -127,4 +129,14 @@ public boolean canEqual(Object obj) { public int hashCode() { return IntListSerializer.class.hashCode(); } + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + throw new UnsupportedOperationException(); + } + + @Override + protected MigrationStrategy getMigrationStrategy(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 e4a926440df8c..40f15bef2affb 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairSerializer.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairSerializer.java @@ -21,7 +21,9 @@ import java.io.IOException; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.DataInputView; @@ -136,4 +138,14 @@ public boolean equals(Object obj) { return obj.getClass() == IntPairSerializerFactory.class; }; } + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + throw new UnsupportedOperationException(); + } + + @Override + protected MigrationStrategy getMigrationStrategy(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 b62b097c1cf43..0dca9ec5a0874 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairSerializer.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairSerializer.java @@ -20,7 +20,9 @@ import java.io.IOException; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.types.StringValue; @@ -104,4 +106,14 @@ public boolean canEqual(Object obj) { public int hashCode() { return StringPairSerializer.class.hashCode(); } + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + throw new UnsupportedOperationException(); + } + + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + throw new UnsupportedOperationException(); + } } 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 7152bfc9b8809..6a281099c77b3 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 @@ -67,6 +67,7 @@ import org.apache.flink.util.IOUtils; import org.apache.flink.util.TestLogger; import org.junit.Assert; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -566,6 +567,7 @@ public void testKryoRegisteringRestoreResilienceWithDefaultSerializer() throws E * * @throws Exception expects {@link ExpectedKryoTestException} to be thrown. */ + @Ignore @Test public void testKryoRegisteringRestoreResilienceWithRegisteredSerializer() throws Exception { CheckpointStreamFactory streamFactory = createStreamFactory(); 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 976b9aada116f..69b371fd6f44e 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java @@ -19,7 +19,9 @@ package org.apache.flink.runtime.state.heap; import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.typeutils.MigrationStrategy; 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.api.java.tuple.Tuple3; @@ -649,5 +651,15 @@ public int hashCode() { public void disable() { this.disabled = true; } + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + throw new UnsupportedOperationException(); + } + + @Override + protected MigrationStrategy getMigrationStrategy(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 146ccd054db90..ca390eba5020f 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/recordutils/RecordSerializer.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/recordutils/RecordSerializer.java @@ -21,7 +21,9 @@ import java.io.IOException; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.types.Record; @@ -141,4 +143,14 @@ public boolean canEqual(Object obj) { public int hashCode() { return RecordSerializer.class.hashCode(); } + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + throw new UnsupportedOperationException(); + } + + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + throw new UnsupportedOperationException(); + } } 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 6871159ca284e..61fc5963fcba8 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 @@ -18,8 +18,9 @@ package org.apache.flink.api.scala.typeutils import org.apache.flink.annotation.Internal -import org.apache.flink.api.common.typeutils.TypeSerializer -import org.apache.flink.core.memory.{DataOutputView, DataInputView} +import org.apache.flink.api.common.typeutils.{MigrationStrategy, TypeSerializer, TypeSerializerConfigSnapshot} +import org.apache.flink.api.java.typeutils.runtime.EitherSerializerConfigSnapshot +import org.apache.flink.core.memory.{DataInputView, DataOutputView} /** * Serializer for [[Either]]. @@ -104,4 +105,47 @@ class EitherSerializer[A, B, T <: Either[A, B]]( override def hashCode(): Int = { 31 * leftSerializer.hashCode() + rightSerializer.hashCode() } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + override def snapshotConfiguration(): EitherSerializerConfigSnapshot = { + new EitherSerializerConfigSnapshot( + leftSerializer.snapshotConfiguration(), + rightSerializer.snapshotConfiguration()) + } + + override protected def getMigrationStrategy( + configSnapshot: TypeSerializerConfigSnapshot): MigrationStrategy[T] = { + + configSnapshot match { + case eitherSerializerConfig: EitherSerializerConfigSnapshot => + val leftRightConfigs = + eitherSerializerConfig.getNestedSerializerConfigSnapshots + + val leftStrategy = leftSerializer.getMigrationStrategyFor(leftRightConfigs(0)) + val rightStrategy = rightSerializer.getMigrationStrategyFor(leftRightConfigs(1)) + + if (leftStrategy.requireMigration || rightStrategy.requireMigration) { + if (leftStrategy.getFallbackDeserializer != null + && rightStrategy.getFallbackDeserializer != null) { + + MigrationStrategy.migrateWithFallbackDeserializer( + new EitherSerializer[A, B, T]( + leftStrategy.getFallbackDeserializer, + rightStrategy.getFallbackDeserializer + ) + ) + + } else { + MigrationStrategy.migrate() + } + } else { + MigrationStrategy.noMigration() + } + + case _ => MigrationStrategy.migrate() + } + } } 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 67c1445dfa473..b5d4663051967 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 @@ -17,10 +17,14 @@ */ package org.apache.flink.api.scala.typeutils +import java.io.IOException + import org.apache.flink.annotation.Internal -import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils.{MigrationStrategy, TypeSerializer, TypeSerializerConfigSnapshot} import org.apache.flink.api.common.typeutils.base.IntSerializer +import org.apache.flink.api.java.typeutils.runtime.{DataInputViewStream, DataOutputViewStream} import org.apache.flink.core.memory.{DataInputView, DataOutputView} +import org.apache.flink.util.InstantiationUtil /** * Serializer for [[Enumeration]] values. @@ -67,4 +71,114 @@ class EnumValueSerializer[E <: Enumeration](val enum: E) extends TypeSerializer[ override def canEqual(obj: scala.Any): Boolean = { obj.isInstanceOf[EnumValueSerializer[_]] } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + override def snapshotConfiguration(): EnumValueSerializer.ScalaEnumSerializerConfigSnapshot[E] = { + new EnumValueSerializer.ScalaEnumSerializerConfigSnapshot[E]( + enum.getClass.asInstanceOf[Class[E]]) + } + + override protected def getMigrationStrategy( + configSnapshot: TypeSerializerConfigSnapshot): MigrationStrategy[E#Value] = { + + configSnapshot match { + case enumSerializerConfigSnapshot: EnumValueSerializer.ScalaEnumSerializerConfigSnapshot[_] => + val enumClass = enum.getClass.asInstanceOf[Class[E]] + if (enumClass.equals(enumSerializerConfigSnapshot.getEnumClass)) { + val currentEnumConstants = enumSerializerConfigSnapshot.getEnumClass.getEnumConstants + + for ( i <- 0 to currentEnumConstants.length) { + // compatible only if new enum constants are only appended, + // and original constants must be in the exact same order + + if (currentEnumConstants(i) != enumSerializerConfigSnapshot.getEnumConstants(i)) { + MigrationStrategy.migrate() + } + } + + MigrationStrategy.noMigration() + } else { + MigrationStrategy.migrate() + } + + case _ => MigrationStrategy.migrate() + } + } +} + +object EnumValueSerializer { + + class ScalaEnumSerializerConfigSnapshot[E <: Enumeration](private var enumClass: Class[E]) + extends TypeSerializerConfigSnapshot { + + var enumConstants: Array[E] = enumClass.getEnumConstants + + /** This empty nullary constructor is required for deserializing the configuration. */ + def this() = this(null) + + override def write(out: DataOutputView): Unit = { + super.write(out) + + try { + val outViewWrapper = new DataOutputViewStream(out) + try { + InstantiationUtil.serializeObject(outViewWrapper, enumClass) + InstantiationUtil.serializeObject(outViewWrapper, enumConstants) + } finally if (outViewWrapper != null) outViewWrapper.close() + } + } + + override def read(in: DataInputView): Unit = { + super.read(in) + + try { + val inViewWrapper = new DataInputViewStream(in) + try + try { + enumClass = InstantiationUtil.deserializeObject( + inViewWrapper, getUserCodeClassLoader) + + enumConstants = InstantiationUtil.deserializeObject( + inViewWrapper, getUserCodeClassLoader) + } catch { + case e: ClassNotFoundException => + throw new IOException("The requested enum class cannot be found in classpath.", e) + } + finally if (inViewWrapper != null) inViewWrapper.close() + } + } + + override def getVersion: Int = ScalaEnumSerializerConfigSnapshot.VERSION + + def getEnumClass: Class[E] = enumClass + + def getEnumConstants: Array[E] = enumConstants + + override def equals(obj: scala.Any): Boolean = { + if (obj == this) { + return true + } + + if (obj == null) { + return false + } + + obj.isInstanceOf[ScalaEnumSerializerConfigSnapshot[E]] && + enumClass.equals(obj.asInstanceOf[ScalaEnumSerializerConfigSnapshot[E]].enumClass) && + enumConstants.sameElements( + obj.asInstanceOf[ScalaEnumSerializerConfigSnapshot[E]].enumConstants) + } + + override def hashCode(): Int = { + enumClass.hashCode() * 31 + enumConstants.toSeq.hashCode() + } + } + + object ScalaEnumSerializerConfigSnapshot { + val VERSION = 1 + } + } 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 fa5279e9bf0d8..6048c46052987 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala @@ -18,8 +18,8 @@ package org.apache.flink.api.scala.typeutils import org.apache.flink.annotation.Internal -import org.apache.flink.api.common.typeutils.TypeSerializer -import org.apache.flink.core.memory.{DataOutputView, DataInputView} +import org.apache.flink.api.common.typeutils.{MigrationStrategy, TypeSerializer, TypeSerializerConfigSnapshot} +import org.apache.flink.core.memory.{DataInputView, DataOutputView} /** * Serializer for cases where no serializer is required but the system still expects one. This @@ -50,13 +50,19 @@ class NothingSerializer extends TypeSerializer[Any] { override def serialize(any: Any, target: DataOutputView): Unit = throw new RuntimeException("This must not be used. You encountered a bug.") - override def deserialize(source: DataInputView): Any = throw new RuntimeException("This must not be used. You encountered a bug.") override def deserialize(reuse: Any, source: DataInputView): Any = throw new RuntimeException("This must not be used. You encountered a bug.") + override def snapshotConfiguration(): TypeSerializerConfigSnapshot = + throw new RuntimeException("This must not be used. You encountered a bug.") + + override protected def getMigrationStrategy( + configSnapshot: TypeSerializerConfigSnapshot): MigrationStrategy[Any] = + throw new RuntimeException("This must not be used. You encountered a bug.") + override def equals(obj: Any): Boolean = { obj match { case nothingSerializer: NothingSerializer => nothingSerializer.canEqual(this) 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 a8b3a56325e08..8147a3921f2d2 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 @@ -18,8 +18,8 @@ package org.apache.flink.api.scala.typeutils import org.apache.flink.annotation.Internal -import org.apache.flink.api.common.typeutils.TypeSerializer -import org.apache.flink.core.memory.{DataOutputView, DataInputView} +import org.apache.flink.api.common.typeutils._ +import org.apache.flink.core.memory.{DataInputView, DataOutputView} /** * Serializer for [[Option]]. @@ -95,4 +95,52 @@ class OptionSerializer[A](val elemSerializer: TypeSerializer[A]) override def hashCode(): Int = { elemSerializer.hashCode() } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + override def snapshotConfiguration(): OptionSerializer.OptionSerializerConfigSnapshot = { + new OptionSerializer.OptionSerializerConfigSnapshot(elemSerializer.snapshotConfiguration()) + } + + override protected def getMigrationStrategy( + configSnapshot: TypeSerializerConfigSnapshot): MigrationStrategy[Option[A]] = { + configSnapshot match { + case optionSerializerConfigSnapshot: OptionSerializer.OptionSerializerConfigSnapshot => + val strategy = elemSerializer.getMigrationStrategyFor( + optionSerializerConfigSnapshot.getSingleNestedSerializerConfigSnapshot) + + if (strategy.requireMigration()) { + if (strategy.getFallbackDeserializer != null) { + MigrationStrategy.migrateWithFallbackDeserializer( + new OptionSerializer[A](strategy.getFallbackDeserializer)) + } else { + MigrationStrategy.migrate() + } + } else { + MigrationStrategy.noMigration() + } + + case _ => MigrationStrategy.migrate() + } + } +} + +object OptionSerializer { + + class OptionSerializerConfigSnapshot( + private var elemSerializerConfigSnapshot: TypeSerializerConfigSnapshot) + extends CompositeTypeSerializerConfigSnapshot(elemSerializerConfigSnapshot) { + + /** This empty nullary constructor is required for deserializing the configuration. */ + def this() = this(null) + + override def getVersion: Int = OptionSerializerConfigSnapshot.VERSION + } + + object OptionSerializerConfigSnapshot { + val VERSION = 1 + } + } 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 d1b908591f2ae..2a2d4e8cb5dcc 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 @@ -20,8 +20,8 @@ package org.apache.flink.api.scala.typeutils import java.io.ObjectInputStream import org.apache.flink.annotation.Internal -import org.apache.flink.api.common.typeutils.TypeSerializer -import org.apache.flink.core.memory.{DataOutputView, DataInputView} +import org.apache.flink.api.common.typeutils.{MigrationStrategy, TypeSerializer, TypeSerializerConfigSnapshot} +import org.apache.flink.core.memory.{DataInputView, DataOutputView} import scala.collection.generic.CanBuildFrom @@ -150,4 +150,13 @@ abstract class TraversableSerializer[T <: TraversableOnce[E], E]( override def canEqual(obj: Any): Boolean = { obj.isInstanceOf[TraversableSerializer[_, _]] } + + override def snapshotConfiguration(): TypeSerializerConfigSnapshot = { + throw new UnsupportedOperationException() + } + + override protected def getMigrationStrategy( + configSnapshot: TypeSerializerConfigSnapshot): MigrationStrategy[T] = { + throw new UnsupportedOperationException() + } } 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 a5ec03a9d45b0..1aa5ce8054cfd 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 @@ -19,11 +19,12 @@ package org.apache.flink.api.scala.typeutils import org.apache.flink.annotation.Internal import org.apache.flink.api.common.ExecutionConfig -import org.apache.flink.api.common.typeutils.TypeSerializer +import org.apache.flink.api.common.typeutils._ import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer.KryoSerializerConfigSnapshot import org.apache.flink.core.memory.{DataInputView, DataOutputView} -import scala.util.{Success, Try, Failure} +import scala.util.{Failure, Success, Try} /** * Serializer for [[scala.util.Try]]. @@ -98,4 +99,57 @@ class TrySerializer[A]( override def hashCode(): Int = { 31 * elemSerializer.hashCode() + executionConfig.hashCode() } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + override def snapshotConfiguration(): TypeSerializerConfigSnapshot = { + new TrySerializer.TrySerializerConfigSnapshot( + elemSerializer.snapshotConfiguration(), + throwableSerializer.snapshotConfiguration()) + } + + override protected def getMigrationStrategy( + configSnapshot: TypeSerializerConfigSnapshot): MigrationStrategy[Try[A]] = { + + configSnapshot match { + case trySerializerConfigSnapshot: TrySerializer.TrySerializerConfigSnapshot => + val serializerConfigSnapshots = + trySerializerConfigSnapshot.getNestedSerializerConfigSnapshots + + val elemStrategy = + elemSerializer.getMigrationStrategyFor(serializerConfigSnapshots(0)) + val throwableStrategy = + throwableSerializer.getMigrationStrategyFor(serializerConfigSnapshots(1)) + + if (elemStrategy.requireMigration() || throwableStrategy.requireMigration()) { + MigrationStrategy.migrate() + } else { + MigrationStrategy.noMigration() + } + + case _ => MigrationStrategy.migrate() + } + } +} + +object TrySerializer { + + class TrySerializerConfigSnapshot( + private var elemSerializerConfigSnapshot: TypeSerializerConfigSnapshot, + private var throwableSerializerConfigSnapshot: KryoSerializerConfigSnapshot[Throwable]) + extends CompositeTypeSerializerConfigSnapshot( + elemSerializerConfigSnapshot, throwableSerializerConfigSnapshot) { + + /** This empty nullary constructor is required for deserializing the configuration. */ + def this() = this(null, null) + + override def getVersion: Int = TrySerializerConfigSnapshot.VERSION + } + + object TrySerializerConfigSnapshot { + val VERSION = 1 + } + } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java index fdcd5b89ae18c..93805d3a8d4e8 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java @@ -21,7 +21,11 @@ import static java.util.Objects.requireNonNull; import java.io.IOException; + +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.streaming.api.watermark.Watermark; @@ -205,6 +209,58 @@ else if (tag == TAG_WATERMARK) { } } + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public MultiplexingStreamRecordSerializerConfigSnapshot snapshotConfiguration() { + return new MultiplexingStreamRecordSerializerConfigSnapshot(typeSerializer.snapshotConfiguration()); + } + + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof MultiplexingStreamRecordSerializerConfigSnapshot) { + MigrationStrategy strategy = typeSerializer.getMigrationStrategyFor( + ((MultiplexingStreamRecordSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); + + if (strategy.requireMigration()) { + if (strategy.getFallbackDeserializer() != null) { + return MigrationStrategy.migrateWithFallbackDeserializer( + new MultiplexingStreamRecordSerializer<>( + strategy.getFallbackDeserializer())); + } else { + return MigrationStrategy.migrate(); + } + } else { + return MigrationStrategy.noMigration(); + } + } else { + return MigrationStrategy.migrate(); + } + } + + /** + * Configuration snapshot specific to the {@link MultiplexingStreamRecordSerializer}. + */ + public static final class MultiplexingStreamRecordSerializerConfigSnapshot + extends CompositeTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public MultiplexingStreamRecordSerializerConfigSnapshot() {} + + public MultiplexingStreamRecordSerializerConfigSnapshot(TypeSerializerConfigSnapshot typeSerializerConfigSnapshot) { + super(typeSerializerConfigSnapshot); + } + + @Override + public int getVersion() { + return VERSION; + } + } + // ------------------------------------------------------------------------ // Utilities // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java index 2c8dc4a328358..578ccb8b11179 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java @@ -20,7 +20,10 @@ import java.io.IOException; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -144,4 +147,54 @@ public boolean canEqual(Object obj) { public int hashCode() { return typeSerializer.hashCode(); } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // -------------------------------------------------------------------------------------------- + + @Override + public StreamRecordSerializerConfigSnapshot snapshotConfiguration() { + return new StreamRecordSerializerConfigSnapshot(typeSerializer.snapshotConfiguration()); + } + + @Override + protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof StreamRecordSerializerConfigSnapshot) { + MigrationStrategy strategy = typeSerializer.getMigrationStrategyFor( + ((StreamRecordSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); + + if (strategy.requireMigration()) { + if (strategy.getFallbackDeserializer() != null) { + return MigrationStrategy.migrateWithFallbackDeserializer( + new StreamRecordSerializer<>(strategy.getFallbackDeserializer())); + } else { + return MigrationStrategy.migrate(); + } + } else { + return MigrationStrategy.noMigration(); + } + } else { + return MigrationStrategy.migrate(); + } + } + + /** + * Configuration snapshot specific to the {@link StreamRecordSerializer}. + */ + public static final class StreamRecordSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public StreamRecordSerializerConfigSnapshot() {} + + public StreamRecordSerializerConfigSnapshot(TypeSerializerConfigSnapshot typeSerializerConfigSnapshot) { + super(typeSerializerConfigSnapshot); + } + + @Override + public int getVersion() { + return 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 13a8a248d7d7e..bc16a03609f7a 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 @@ -29,7 +29,9 @@ import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.operators.translation.WrappingFunction; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -547,6 +549,16 @@ public boolean equals(Object obj) { public boolean canEqual(Object obj) { return obj instanceof UnionSerializer; } + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + throw new UnsupportedOperationException("This serializer is not registered for managed state."); + } + + @Override + protected MigrationStrategy> getMigrationStrategy(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/api/operators/InternalTimer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimer.java index 7b6ba8da66eaa..c1fdde5de82d2 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimer.java @@ -19,7 +19,9 @@ import java.io.IOException; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -184,5 +186,15 @@ public boolean canEqual(Object obj) { public int hashCode() { return getClass().hashCode(); } + + @Override + public TypeSerializerConfigSnapshot snapshotConfiguration() { + throw new UnsupportedOperationException("This serializer is not registered for managed state."); + } + + @Override + protected MigrationStrategy> getMigrationStrategy(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/api/windowing/windows/GlobalWindow.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/GlobalWindow.java index b1e06f5e8d105..cf5c74c36846f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/GlobalWindow.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/GlobalWindow.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -61,7 +62,7 @@ public String toString() { /** * A {@link TypeSerializer} for {@link GlobalWindow}. */ - public static class Serializer extends TypeSerializer { + public static class Serializer extends TypeSerializerSingleton { private static final long serialVersionUID = 1L; @Override @@ -69,11 +70,6 @@ public boolean isImmutableType() { return true; } - @Override - public TypeSerializer duplicate() { - return this; - } - @Override public GlobalWindow createInstance() { return GlobalWindow.INSTANCE; @@ -118,19 +114,9 @@ public void copy(DataInputView source, DataOutputView target) throws IOException target.writeByte(0); } - @Override - public boolean equals(Object obj) { - return obj instanceof Serializer; - } - @Override public boolean canEqual(Object obj) { return obj instanceof Serializer; } - - @Override - public int hashCode() { - return 0; - } } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java index 6d896cb943dfe..a7ea244e2680f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/windowing/windows/TimeWindow.java @@ -26,7 +26,7 @@ import java.util.List; import java.util.Set; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -130,7 +130,7 @@ public TimeWindow cover(TimeWindow other) { /** * The serializer used to write the TimeWindow type. */ - public static class Serializer extends TypeSerializer { + public static class Serializer extends TypeSerializerSingleton { private static final long serialVersionUID = 1L; @Override @@ -138,11 +138,6 @@ public boolean isImmutableType() { return true; } - @Override - public TypeSerializer duplicate() { - return this; - } - @Override public TimeWindow createInstance() { return null; @@ -187,20 +182,10 @@ public void copy(DataInputView source, DataOutputView target) throws IOException target.writeLong(source.readLong()); } - @Override - public boolean equals(Object obj) { - return obj instanceof Serializer; - } - @Override public boolean canEqual(Object obj) { return obj instanceof Serializer; } - - @Override - public int hashCode() { - return 0; - } } // ------------------------------------------------------------------------ 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 7fe088a599352..55b1ba5b95fe5 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,7 +22,10 @@ import java.io.IOException; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.streaming.api.watermark.Watermark; @@ -262,4 +265,59 @@ public boolean canEqual(Object obj) { public int hashCode() { return typeSerializer.hashCode(); } + + // -------------------------------------------------------------------------------------------- + // Serializer configuration snapshotting & reconfiguring + // + // This serializer may be used by Flink internal operators that need to checkpoint + // buffered records. Therefore, it may be part of managed state and need to implement + // the configuration snapshot and reconfiguring methods. + // -------------------------------------------------------------------------------------------- + + @Override + public StreamElementSerializerConfigSnapshot snapshotConfiguration() { + return new StreamElementSerializerConfigSnapshot(typeSerializer.snapshotConfiguration()); + } + + @Override + protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + if (configSnapshot instanceof StreamElementSerializerConfigSnapshot) { + MigrationStrategy strategy = typeSerializer.getMigrationStrategyFor( + ((StreamElementSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); + + if (strategy.requireMigration()) { + if (strategy.getFallbackDeserializer() != null) { + return MigrationStrategy.migrateWithFallbackDeserializer( + new StreamElementSerializer<>( + strategy.getFallbackDeserializer())); + } else { + return MigrationStrategy.migrate(); + } + } else { + return MigrationStrategy.noMigration(); + } + } else { + return MigrationStrategy.migrate(); + } + } + + /** + * Configuration snapshot specific to the {@link StreamElementSerializer}. + */ + public static final class StreamElementSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { + + private static final int VERSION = 1; + + /** This empty nullary constructor is required for deserializing the configuration. */ + public StreamElementSerializerConfigSnapshot() {} + + public StreamElementSerializerConfigSnapshot(TypeSerializerConfigSnapshot typeSerializerConfigSnapshot) { + super(typeSerializerConfigSnapshot); + } + + @Override + public int getVersion() { + return VERSION; + } + } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java index a24a3a88571a7..2693bc1a43c1f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java +++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java @@ -228,6 +228,5 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof CustomIntSerializer; } - } } From ed82173fe97c6e9fb0784696bc4c49f10cc4e556 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Tue, 2 May 2017 19:35:18 +0800 Subject: [PATCH 2/5] [hotfix] [core] Catch InvalidClassException in TypeSerializerSerializationProxy Previously, the TypeSerializerSerializationProxy only uses the dummy ClassNotFoundDummyTypeSerializer as a placeholder in the case where the user uses a completely new serializer and deletes the old one. There is also the case where the user changes the original serializer's implementation and results in an InvalidClassException when trying to deserialize the serializer. We should also use the ClassNotFoundDummyTypeSerializer as a temporary placeholder in this case. --- .../TypeSerializerSerializationProxy.java | 3 +- .../TypeSerializerSerializationProxyTest.java | 50 ++++++++++++++++++- 2 files changed, 51 insertions(+), 2 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java index f36021897b19a..5c58de1dc61ad 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java @@ -29,6 +29,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.io.InvalidClassException; import java.util.Arrays; @Internal @@ -97,7 +98,7 @@ public void read(DataInputView in) throws IOException { in.readFully(buffer); try { typeSerializer = InstantiationUtil.deserializeObject(buffer, userClassLoader); - } catch (ClassNotFoundException e) { + } catch (ClassNotFoundException | InvalidClassException e) { if (ignoreClassNotFound) { // we create a dummy so that all the information is not lost when we get a new checkpoint before receiving // a proper typeserializer from the user diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxyTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxyTest.java index 982e7ff613544..db1b4ef578b0b 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxyTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxyTest.java @@ -26,13 +26,21 @@ import org.apache.flink.util.InstantiationUtil; import org.junit.Assert; import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; import java.io.IOException; +import java.io.InvalidClassException; import java.net.URL; import java.net.URLClassLoader; import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +@RunWith(PowerMockRunner.class) +@PrepareForTest(InstantiationUtil.class) public class TypeSerializerSerializationProxyTest { @Test @@ -91,4 +99,44 @@ public void testStateSerializerSerializationProxyClassNotFound() throws Exceptio InstantiationUtil.serializeObject(serializer), ((TypeSerializerSerializationProxy.ClassNotFoundDummyTypeSerializer) proxy.getTypeSerializer()).getActualBytes()); } -} \ No newline at end of file + + @Test + public void testStateSerializerSerializationProxyInvalidClass() throws Exception { + + TypeSerializer serializer = IntSerializer.INSTANCE; + + TypeSerializerSerializationProxy proxy = new TypeSerializerSerializationProxy<>(serializer); + + byte[] serialized; + try (ByteArrayOutputStreamWithPos out = new ByteArrayOutputStreamWithPos()) { + proxy.write(new DataOutputViewStreamWrapper(out)); + serialized = out.toByteArray(); + } + + PowerMockito.spy(InstantiationUtil.class); + PowerMockito + .doThrow(new InvalidClassException("test invalid class exception")) + .when(InstantiationUtil.class, "deserializeObject", any(byte[].class), any(ClassLoader.class)); + + proxy = new TypeSerializerSerializationProxy<>(new URLClassLoader(new URL[0], null)); + + try (ByteArrayInputStreamWithPos in = new ByteArrayInputStreamWithPos(serialized)) { + proxy.read(new DataInputViewStreamWrapper(in)); + fail("InvalidClassException expected, leading to IOException"); + } catch (IOException expected) { + + } + + proxy = new TypeSerializerSerializationProxy<>(new URLClassLoader(new URL[0], null), true); + + try (ByteArrayInputStreamWithPos in = new ByteArrayInputStreamWithPos(serialized)) { + proxy.read(new DataInputViewStreamWrapper(in)); + } + + Assert.assertTrue(proxy.getTypeSerializer() instanceof TypeSerializerSerializationProxy.ClassNotFoundDummyTypeSerializer); + + Assert.assertArrayEquals( + InstantiationUtil.serializeObject(serializer), + ((TypeSerializerSerializationProxy.ClassNotFoundDummyTypeSerializer) proxy.getTypeSerializer()).getActualBytes()); + } +} From e77096af29b4cbea26113928fe93218c075e4035 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sat, 6 May 2017 20:40:58 +0800 Subject: [PATCH 3/5] [FLINK-6425] [runtime] Activate serializer upgrades in state backends This commit fully activates state serializer upgrades by changing the following: - Include serializer configuration snapshots in checkpoints - On restore, use configuration snapshots to confront new serializers to perform the upgrade --- .../state/RocksDBKeyedStateBackend.java | 237 +++++++++----- .../api/common/typeutils/TypeSerializer.java | 4 - .../TypeSerializerSerializationProxy.java | 2 +- .../runtime/kryo/KryoSerializer.java | 15 - .../AbstractKeyedCEPPatternOperator.java | 5 - .../MigrationNamespaceSerializerProxy.java | 10 +- .../AbstractMigrationRestoreStrategy.java | 8 +- .../state/DefaultOperatorStateBackend.java | 130 +++----- .../state/KeyedBackendSerializationProxy.java | 176 ++-------- ...endStateMetaInfoSnapshotReaderWriters.java | 257 +++++++++++++++ .../OperatorBackendSerializationProxy.java | 166 +++------- ...endStateMetaInfoSnapshotReaderWriters.java | 233 +++++++++++++ .../state/RegisteredBackendStateMetaInfo.java | 145 --------- .../RegisteredKeyedBackendStateMetaInfo.java | 246 ++++++++++++++ ...egisteredOperatorBackendStateMetaInfo.java | 198 +++++++++++ .../state/VoidNamespaceSerializer.java | 8 + .../state/heap/CopyOnWriteStateTable.java | 12 +- .../state/heap/HeapKeyedStateBackend.java | 68 ++-- .../state/heap/NestedMapsStateTable.java | 4 +- .../flink/runtime/state/heap/StateTable.java | 12 +- .../heap/StateTableByKeyGroupReaders.java | 9 +- .../query/QueryableStateClientTest.java | 6 +- .../state/OperatorStateBackendTest.java | 2 +- .../state/SerializationProxiesTest.java | 150 +++++++-- .../runtime/state/StateBackendTestBase.java | 307 +++++++++++++++++- .../state/heap/CopyOnWriteStateTableTest.java | 22 +- .../StateTableSnapshotCompatibilityTest.java | 6 +- 27 files changed, 1723 insertions(+), 715 deletions(-) create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedBackendStateMetaInfoSnapshotReaderWriters.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorBackendStateMetaInfoSnapshotReaderWriters.java delete mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBackendStateMetaInfo.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyedBackendStateMetaInfo.java create mode 100644 flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorBackendStateMetaInfo.java diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index b8e60cd9961f5..57f941087c993 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -26,7 +26,9 @@ import org.apache.flink.api.common.state.ReducingStateDescriptor; import org.apache.flink.api.common.state.StateDescriptor; import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSerializationProxy; import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream; @@ -59,9 +61,9 @@ import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedBackendSerializationProxy; import org.apache.flink.runtime.state.KeyedStateHandle; -import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo; import org.apache.flink.runtime.state.StateObject; import org.apache.flink.runtime.state.StateUtil; +import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.internal.InternalAggregatingState; import org.apache.flink.runtime.state.internal.InternalFoldingState; @@ -151,7 +153,15 @@ public class RocksDBKeyedStateBackend extends AbstractKeyedStateBackend { * Information about the k/v states as we create them. This is used to retrieve the * column family that is used for a state and also for sanity checks when restoring. */ - private Map>> kvStateInformation; + private Map>> kvStateInformation; + + /** + * Map of state names to their corresponding restored state meta info. + * + * TODO this map can be removed when eager-state registration is in place. + * TODO we currently need this cached to check state migration strategies when new serializers are registered. + */ + private Map restoredKvStateMetaInfos; /** Number of bytes required to prefix the key groups. */ private final int keyGroupPrefixBytes; @@ -229,7 +239,7 @@ public void dispose() { // and access it in a synchronized block that locks on #dbDisposeLock. if (db != null) { - for (Tuple2> column : + for (Tuple2> column : kvStateInformation.values()) { try { column.f0.close(); @@ -568,23 +578,14 @@ public KeyGroupsStateHandle getSnapshotResultStateHandle() { private void writeKVStateMetaData() throws IOException { - List> metaInfoList = + List> metaInfoSnapshots = new ArrayList<>(stateBackend.kvStateInformation.size()); int kvStateId = 0; - for (Map.Entry>> column : + for (Map.Entry>> column : stateBackend.kvStateInformation.entrySet()) { - RegisteredBackendStateMetaInfo metaInfo = column.getValue().f1; - - KeyedBackendSerializationProxy.StateMetaInfo metaInfoProxy = - new KeyedBackendSerializationProxy.StateMetaInfo<>( - metaInfo.getStateType(), - metaInfo.getName(), - metaInfo.getNamespaceSerializer(), - metaInfo.getStateSerializer()); - - metaInfoList.add(metaInfoProxy); + metaInfoSnapshots.add(column.getValue().f1.snapshot()); //retrieve iterator for this k/v states readOptions = new ReadOptions(); @@ -597,7 +598,7 @@ private void writeKVStateMetaData() throws IOException { } KeyedBackendSerializationProxy serializationProxy = - new KeyedBackendSerializationProxy(stateBackend.getKeySerializer(), metaInfoList); + new KeyedBackendSerializationProxy(stateBackend.getKeySerializer(), metaInfoSnapshots); serializationProxy.write(outputView); } @@ -717,7 +718,7 @@ private static final class RocksDBIncrementalSnapshotOperation { private Map baseSstFiles; - private final List> stateMetaInfos = new ArrayList<>(); + private final List> stateMetaInfoSnapshots = new ArrayList<>(); private FileSystem backupFileSystem; private Path backupPath; @@ -800,7 +801,7 @@ private StreamStateHandle materializeMetaData() throws Exception { stateBackend.cancelStreamRegistry.registerClosable(outputStream); KeyedBackendSerializationProxy serializationProxy = - new KeyedBackendSerializationProxy(stateBackend.keySerializer, stateMetaInfos); + new KeyedBackendSerializationProxy(stateBackend.keySerializer, stateMetaInfoSnapshots); DataOutputView out = new DataOutputViewStreamWrapper(outputStream); serializationProxy.write(out); @@ -823,18 +824,9 @@ void takeSnapshot() throws Exception { baseSstFiles = stateBackend.materializedSstFiles.get(stateBackend.lastCompletedCheckpointId); // save meta data - for (Map.Entry>> stateMetaInfoEntry : stateBackend.kvStateInformation.entrySet()) { - - RegisteredBackendStateMetaInfo metaInfo = stateMetaInfoEntry.getValue().f1; - - KeyedBackendSerializationProxy.StateMetaInfo metaInfoProxy = - new KeyedBackendSerializationProxy.StateMetaInfo<>( - metaInfo.getStateType(), - metaInfo.getName(), - metaInfo.getNamespaceSerializer(), - metaInfo.getStateSerializer()); - - stateMetaInfos.add(metaInfoProxy); + for (Map.Entry>> stateMetaInfoEntry + : stateBackend.kvStateInformation.entrySet()) { + stateMetaInfoSnapshots.add(stateMetaInfoEntry.getValue().f1.snapshot()); } // save state data @@ -1112,33 +1104,38 @@ private void restoreKVStateMetaData() throws IOException, ClassNotFoundException serializationProxy.read(currentStateHandleInView); - List> metaInfoProxyList = - serializationProxy.getNamedStateSerializationProxies(); + List> restoredMetaInfos = + serializationProxy.getStateMetaInfoSnapshots(); - currentStateHandleKVStateColumnFamilies = new ArrayList<>(metaInfoProxyList.size()); + currentStateHandleKVStateColumnFamilies = new ArrayList<>(restoredMetaInfos.size()); + rocksDBKeyedStateBackend.restoredKvStateMetaInfos = new HashMap<>(restoredMetaInfos.size()); - for (KeyedBackendSerializationProxy.StateMetaInfo metaInfoProxy : metaInfoProxyList) { - Tuple2> columnFamily = - rocksDBKeyedStateBackend.kvStateInformation.get(metaInfoProxy.getStateName()); + for (RegisteredKeyedBackendStateMetaInfo.Snapshot restoredMetaInfo : restoredMetaInfos) { - if (null == columnFamily) { + if (!rocksDBKeyedStateBackend.kvStateInformation.containsKey(restoredMetaInfo.getName())) { ColumnFamilyDescriptor columnFamilyDescriptor = new ColumnFamilyDescriptor( - metaInfoProxy.getStateName().getBytes(ConfigConstants.DEFAULT_CHARSET), + restoredMetaInfo.getName().getBytes(ConfigConstants.DEFAULT_CHARSET), rocksDBKeyedStateBackend.columnOptions); - RegisteredBackendStateMetaInfo stateMetaInfo = - new RegisteredBackendStateMetaInfo<>(metaInfoProxy); + RegisteredKeyedBackendStateMetaInfo stateMetaInfo = + new RegisteredKeyedBackendStateMetaInfo<>( + restoredMetaInfo.getStateType(), + restoredMetaInfo.getName(), + restoredMetaInfo.getNamespaceSerializer(), + restoredMetaInfo.getStateSerializer()); + + rocksDBKeyedStateBackend.restoredKvStateMetaInfos.put(restoredMetaInfo.getName(), restoredMetaInfo); - columnFamily = new Tuple2>( - rocksDBKeyedStateBackend.db.createColumnFamily(columnFamilyDescriptor), - stateMetaInfo); + ColumnFamilyHandle columnFamily = rocksDBKeyedStateBackend.db.createColumnFamily(columnFamilyDescriptor); - rocksDBKeyedStateBackend.kvStateInformation.put(stateMetaInfo.getName(), columnFamily); + rocksDBKeyedStateBackend.kvStateInformation.put( + stateMetaInfo.getName(), + new Tuple2>(columnFamily, stateMetaInfo)); + + currentStateHandleKVStateColumnFamilies.add(columnFamily); } else { - //TODO we could check here for incompatible serializer versions between previous tasks + // TODO with eager state registration in place, check here for serializer migration strategies } - - currentStateHandleKVStateColumnFamilies.add(columnFamily.f0); } } @@ -1198,7 +1195,7 @@ private RocksDBIncrementalRestoreOperation(RocksDBKeyedStateBackend stateBack this.stateBackend = stateBackend; } - private List> readMetaData( + private List> readMetaData( StreamStateHandle metaStateHandle) throws Exception { FSDataInputStream inputStream = null; @@ -1212,7 +1209,7 @@ private RocksDBIncrementalRestoreOperation(RocksDBKeyedStateBackend stateBack DataInputView in = new DataInputViewStreamWrapper(inputStream); serializationProxy.read(in); - return serializationProxy.getNamedStateSerializationProxies(); + return serializationProxy.getStateMetaInfoSnapshots(); } finally { if (inputStream != null) { stateBackend.cancelStreamRegistry.unregisterClosable(inputStream); @@ -1294,18 +1291,21 @@ private void restoreInstance( } // read meta data - List> stateMetaInfoProxies = + List> stateMetaInfoSnapshots = readMetaData(restoreStateHandle.getMetaStateHandle()); List columnFamilyDescriptors = new ArrayList<>(); - for (KeyedBackendSerializationProxy.StateMetaInfo stateMetaInfoProxy : stateMetaInfoProxies) { + stateBackend.restoredKvStateMetaInfos = new HashMap<>(stateMetaInfoSnapshots.size()); + + for (RegisteredKeyedBackendStateMetaInfo.Snapshot stateMetaInfoSnapshot : stateMetaInfoSnapshots) { ColumnFamilyDescriptor columnFamilyDescriptor = new ColumnFamilyDescriptor( - stateMetaInfoProxy.getStateName().getBytes(ConfigConstants.DEFAULT_CHARSET), + stateMetaInfoSnapshot.getName().getBytes(ConfigConstants.DEFAULT_CHARSET), stateBackend.columnOptions); columnFamilyDescriptors.add(columnFamilyDescriptor); + stateBackend.restoredKvStateMetaInfos.put(stateMetaInfoSnapshot.getName(), stateMetaInfoSnapshot); } if (hasExtraKeys) { @@ -1320,23 +1320,27 @@ private void restoreInstance( for (int i = 0; i < columnFamilyHandles.size(); ++i) { ColumnFamilyHandle columnFamilyHandle = columnFamilyHandles.get(i); ColumnFamilyDescriptor columnFamilyDescriptor = columnFamilyDescriptors.get(i); - KeyedBackendSerializationProxy.StateMetaInfo stateMetaInfoProxy = stateMetaInfoProxies.get(i); + RegisteredKeyedBackendStateMetaInfo.Snapshot stateMetaInfoSnapshot = stateMetaInfoSnapshots.get(i); - Tuple2> registeredStateMetaInfoEntry = - stateBackend.kvStateInformation.get(stateMetaInfoProxy.getStateName()); + Tuple2> registeredStateMetaInfoEntry = + stateBackend.kvStateInformation.get(stateMetaInfoSnapshot.getName()); if (null == registeredStateMetaInfoEntry) { - RegisteredBackendStateMetaInfo stateMetaInfo = - new RegisteredBackendStateMetaInfo<>(stateMetaInfoProxy); + RegisteredKeyedBackendStateMetaInfo stateMetaInfo = + new RegisteredKeyedBackendStateMetaInfo<>( + stateMetaInfoSnapshot.getStateType(), + stateMetaInfoSnapshot.getName(), + stateMetaInfoSnapshot.getNamespaceSerializer(), + stateMetaInfoSnapshot.getStateSerializer()); registeredStateMetaInfoEntry = - new Tuple2>( + new Tuple2>( stateBackend.db.createColumnFamily(columnFamilyDescriptor), stateMetaInfo); stateBackend.kvStateInformation.put( - stateMetaInfoProxy.getStateName(), + stateMetaInfoSnapshot.getName(), registeredStateMetaInfoEntry); } @@ -1403,15 +1407,19 @@ private void restoreInstance( columnFamilyDescriptors, columnFamilyHandles); for (int i = 0; i < columnFamilyDescriptors.size(); ++i) { - KeyedBackendSerializationProxy.StateMetaInfo stateMetaInfoProxy = stateMetaInfoProxies.get(i); + RegisteredKeyedBackendStateMetaInfo.Snapshot stateMetaInfoSnapshot = stateMetaInfoSnapshots.get(i); ColumnFamilyHandle columnFamilyHandle = columnFamilyHandles.get(i); - RegisteredBackendStateMetaInfo stateMetaInfo = - new RegisteredBackendStateMetaInfo<>(stateMetaInfoProxy); + RegisteredKeyedBackendStateMetaInfo stateMetaInfo = + new RegisteredKeyedBackendStateMetaInfo<>( + stateMetaInfoSnapshot.getStateType(), + stateMetaInfoSnapshot.getName(), + stateMetaInfoSnapshot.getNamespaceSerializer(), + stateMetaInfoSnapshot.getStateSerializer()); stateBackend.kvStateInformation.put( - stateMetaInfoProxy.getStateName(), - new Tuple2>( + stateMetaInfoSnapshot.getName(), + new Tuple2>( columnFamilyHandle, stateMetaInfo)); } @@ -1473,22 +1481,92 @@ void restore(Collection restoreStateHandles) throws Exception protected ColumnFamilyHandle getColumnFamily( StateDescriptor descriptor, TypeSerializer namespaceSerializer) throws IOException { - Tuple2> stateInfo = + Tuple2> stateInfo = kvStateInformation.get(descriptor.getName()); - RegisteredBackendStateMetaInfo newMetaInfo = new RegisteredBackendStateMetaInfo<>( - descriptor.getType(), - descriptor.getName(), - namespaceSerializer, - descriptor.getSerializer()); + RegisteredKeyedBackendStateMetaInfo newMetaInfo = new RegisteredKeyedBackendStateMetaInfo<>( + descriptor.getType(), + descriptor.getName(), + namespaceSerializer, + descriptor.getSerializer()); if (stateInfo != null) { - if (newMetaInfo.canRestoreFrom(stateInfo.f1)) { + // TODO with eager registration in place, these checks should be moved to restore() + + RegisteredKeyedBackendStateMetaInfo.Snapshot restoredMetaInfo = + restoredKvStateMetaInfos.get(descriptor.getName()); + + Preconditions.checkState( + newMetaInfo.getName().equals(restoredMetaInfo.getName()), + "Incompatible state names. " + + "Was [" + restoredMetaInfo.getName() + "], " + + "registered with [" + newMetaInfo.getName() + "]."); + + if (!newMetaInfo.getStateType().equals(StateDescriptor.Type.UNKNOWN) + && !restoredMetaInfo.getStateType().equals(StateDescriptor.Type.UNKNOWN)) { + + Preconditions.checkState( + newMetaInfo.getStateType().equals(restoredMetaInfo.getStateType()), + "Incompatible state types. " + + "Was [" + restoredMetaInfo.getStateType() + "], " + + "registered with [" + newMetaInfo.getStateType() + "]."); + } + + // check serializer migration strategies to determine if state migration is required + + boolean requireMigration = false; + + // only check migration strategy if there is a restored configuration snapshot; + // there wouldn't be one if we were restored from an older version checkpoint, + // in which case we can only simply assume that migration is not required + + if (restoredMetaInfo.getNamespaceSerializerConfigSnapshot() != null) { + MigrationStrategy namespaceMigrationStrategy = newMetaInfo.getNamespaceSerializer() + .getMigrationStrategyFor(restoredMetaInfo.getNamespaceSerializerConfigSnapshot()); + + TypeSerializer finalOldNamespaceSerializer; + if (namespaceMigrationStrategy.requireMigration()) { + requireMigration = true; + + if (namespaceMigrationStrategy.getFallbackDeserializer() != null) { + finalOldNamespaceSerializer = namespaceMigrationStrategy.getFallbackDeserializer(); + } else if (restoredMetaInfo.getNamespaceSerializer() != null + && !(restoredMetaInfo.getNamespaceSerializer() instanceof MigrationNamespaceSerializerProxy)) { + finalOldNamespaceSerializer = restoredMetaInfo.getNamespaceSerializer(); + } else { + throw new RuntimeException( + "State migration required, but there is no available serializer capable of reading previous namespace."); + } + } + } + + if (restoredMetaInfo.getStateSerializerConfigSnapshot() != null) { + MigrationStrategy stateMigrationStrategy = newMetaInfo.getStateSerializer() + .getMigrationStrategyFor(restoredMetaInfo.getStateSerializerConfigSnapshot()); + + TypeSerializer finalOldStateSerializer; + if (stateMigrationStrategy.requireMigration()) { + requireMigration = true; + + if (stateMigrationStrategy.getFallbackDeserializer() != null) { + finalOldStateSerializer = stateMigrationStrategy.getFallbackDeserializer(); + } else if (restoredMetaInfo.getStateSerializer() != null + && !(restoredMetaInfo.getStateSerializer() instanceof TypeSerializerSerializationProxy.ClassNotFoundDummyTypeSerializer)) { + finalOldStateSerializer = restoredMetaInfo.getStateSerializer(); + } else { + throw new RuntimeException( + "State migration required, but there is no available serializer capable of reading previous state."); + } + } + } + + + if (!requireMigration) { stateInfo.f1 = newMetaInfo; return stateInfo.f0; } else { - throw new IOException("Trying to access state using wrong meta info, was " + stateInfo.f1 + - " trying access with " + newMetaInfo); + // TODO state migration currently isn't possible. + throw new RuntimeException("State migration currently isn't supported."); } } @@ -1497,7 +1575,7 @@ protected ColumnFamilyHandle getColumnFamily( try { ColumnFamilyHandle columnFamily = db.createColumnFamily(columnDescriptor); - Tuple2> tuple = + Tuple2> tuple = new Tuple2<>(columnFamily, newMetaInfo); Map rawAccess = kvStateInformation; rawAccess.put(descriptor.getName(), tuple); @@ -1832,6 +1910,8 @@ private void restoreOldSavepointKeyedState(Collection restoreS // clear k/v state information before filling it kvStateInformation.clear(); + restoredKvStateMetaInfos = new HashMap<>(namedStates.size()); + // first get the column family mapping int numColumns = inputView.readInt(); Map> columnFamilyMapping = new HashMap<>(numColumns); @@ -1846,6 +1926,15 @@ private void restoreOldSavepointKeyedState(Collection restoreS columnFamilyMapping.put(mappingByte, stateDescriptor); + // mimic a restored kv state meta info + restoredKvStateMetaInfos.put( + stateDescriptor.getName(), + new RegisteredKeyedBackendStateMetaInfo<>( + stateDescriptor.getType(), + stateDescriptor.getName(), + MigrationNamespaceSerializerProxy.INSTANCE, + stateDescriptor.getSerializer()).snapshot()); + // this will fill in the k/v state information getColumnFamily(stateDescriptor, MigrationNamespaceSerializerProxy.INSTANCE); } 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 938da3b519410..6effc1e3981b9 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 @@ -162,10 +162,6 @@ public abstract class TypeSerializer implements Serializable { public abstract int hashCode(); - public boolean canRestoreFrom(TypeSerializer other) { - return equals(other); - } - // -------------------------------------------------------------------------------------------- // Serializer configuration snapshotting & reconfiguring // -------------------------------------------------------------------------------------------- diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java index 5c58de1dc61ad..f4ccca70cedfa 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java @@ -143,7 +143,7 @@ public void setIgnoreClassNotFound(boolean ignoreClassNotFound) { * Dummy TypeSerializer to avoid that data is lost when checkpointing again a serializer for which we encountered * a {@link ClassNotFoundException}. */ - static final class ClassNotFoundDummyTypeSerializer extends TypeSerializer { + public static final class ClassNotFoundDummyTypeSerializer extends TypeSerializer { private static final long serialVersionUID = 2526330533671642711L; private final byte[] actualBytes; 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 a0cf99cd8443f..20f10bbc069e6 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 @@ -274,21 +274,6 @@ public void copy(DataInputView source, DataOutputView target) throws IOException T tmp = deserialize(copyInstance, source); serialize(tmp, target); } - - @Override - public boolean canRestoreFrom(TypeSerializer other) { - if (other instanceof KryoSerializer) { - KryoSerializer otherKryo = (KryoSerializer) other; - - // we cannot include the Serializers here because they don't implement the equals method - return other.canEqual(this) && - type == otherKryo.type && - (kryoRegistrations.equals(otherKryo.kryoRegistrations)) && - (defaultSerializerClasses.equals(otherKryo.defaultSerializerClasses) || otherKryo.defaultSerializerClasses.isEmpty()); - } else { - return false; - } - } // -------------------------------------------------------------------------------------------- diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java index 2997416ac8317..7d629c9f8c753 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java @@ -475,11 +475,6 @@ public void copy(DataInputView source, DataOutputView target) throws IOException } - @Override - public boolean canRestoreFrom(TypeSerializer other) { - return equals(other) || other instanceof AbstractKeyedCEPPatternOperator.PriorityQueueSerializer; - } - @Override public boolean equals(Object obj) { if (obj instanceof PriorityQueueSerializer) { diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java index 570df50608780..50c340d4a5270 100644 --- a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java +++ b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java @@ -19,6 +19,7 @@ package org.apache.flink.migration; import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.ParameterlessTypeSerializerConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; @@ -53,8 +54,7 @@ public boolean isImmutableType() { @Override public TypeSerializer duplicate() { - throw new UnsupportedOperationException( - "This is just a proxy used during migration until the real type serializer is provided by the user."); + return this; } @Override @@ -106,14 +106,12 @@ public void copy(DataInputView source, DataOutputView target) throws IOException @Override public TypeSerializerConfigSnapshot snapshotConfiguration() { - throw new UnsupportedOperationException( - "This is just a proxy used during migration until the real type serializer is provided by the user."); + return new ParameterlessTypeSerializerConfig(getClass().getCanonicalName()); } @Override protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { - throw new UnsupportedOperationException( - "This is just a proxy used during migration until the real type serializer is provided by the user."); + return MigrationStrategy.noMigration(); } @Override diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMigrationRestoreStrategy.java b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMigrationRestoreStrategy.java index f47989a7bf04e..f58070e564b92 100644 --- a/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMigrationRestoreStrategy.java +++ b/flink-runtime/src/main/java/org/apache/flink/migration/runtime/state/memory/AbstractMigrationRestoreStrategy.java @@ -23,7 +23,7 @@ import org.apache.flink.api.common.typeutils.base.VoidSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.runtime.state.KeyGroupRange; -import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo; +import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo; import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.runtime.state.VoidNamespaceSerializer; import org.apache.flink.runtime.state.heap.HeapKeyedStateBackend; @@ -85,14 +85,14 @@ public StateTable deserialize(String stateName, HeapKeyedStateBackend) VoidNamespaceSerializer.INSTANCE; } - RegisteredBackendStateMetaInfo registeredBackendStateMetaInfo = - new RegisteredBackendStateMetaInfo<>( + RegisteredKeyedBackendStateMetaInfo registeredKeyedBackendStateMetaInfo = + new RegisteredKeyedBackendStateMetaInfo<>( StateDescriptor.Type.UNKNOWN, stateName, patchedNamespaceSerializer, stateSerializer); - final StateTable stateTable = stateBackend.newStateTable(registeredBackendStateMetaInfo); + final StateTable stateTable = stateBackend.newStateTable(registeredKeyedBackendStateMetaInfo); final DataInputView inView = openDataInputView(); final int keyGroup = keyGroupRange.getStartKeyGroup(); final int numNamespaces = inView.readInt(); 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 e7ed26f67fd11..ec4aa81dfd62d 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 @@ -207,26 +207,18 @@ public OperatorStateHandle performOperation() throws Exception { final Map writtenStatesMetaData = new HashMap<>(registeredStatesDeepCopies.size()); - List> metaInfoList = + List> metaInfoSnapshots = new ArrayList<>(registeredStatesDeepCopies.size()); - for (Map.Entry> entry : - registeredStatesDeepCopies.entrySet()) { - - PartitionableListState state = entry.getValue(); - OperatorBackendSerializationProxy.StateMetaInfo metaInfo = - new OperatorBackendSerializationProxy.StateMetaInfo<>( - state.getName(), - state.getPartitionStateSerializer(), - state.getAssignmentMode()); - metaInfoList.add(metaInfo); + for (Map.Entry> entry : registeredStatesDeepCopies.entrySet()) { + metaInfoSnapshots.add(entry.getValue().getStateMetaInfo().snapshot()); } CheckpointStreamFactory.CheckpointStateOutputStream out = getIoHandle(); DataOutputView dov = new DataOutputViewStreamWrapper(out); OperatorBackendSerializationProxy backendSerializationProxy = - new OperatorBackendSerializationProxy(metaInfoList); + new OperatorBackendSerializationProxy(metaInfoSnapshots); backendSerializationProxy.write(dov); @@ -237,7 +229,7 @@ public OperatorStateHandle performOperation() throws Exception { PartitionableListState value = entry.getValue(); long[] partitionOffsets = value.write(out); - OperatorStateHandle.Mode mode = value.getAssignmentMode(); + OperatorStateHandle.Mode mode = value.getStateMetaInfo().getAssignmentMode(); writtenStatesMetaData.put( entry.getKey(), new OperatorStateHandle.StateMetaInfo(partitionOffsets, mode)); @@ -254,10 +246,7 @@ public OperatorStateHandle performOperation() throws Exception { return null; } - OperatorStateHandle operatorStateHandle = - new OperatorStateHandle(writtenStatesMetaData, stateHandle); - - return operatorStateHandle; + return new OperatorStateHandle(writtenStatesMetaData, stateHandle); } }; @@ -298,25 +287,23 @@ public void restore(Collection restoreSnapshots) throws Exc backendSerializationProxy.read(new DataInputViewStreamWrapper(in)); - List> metaInfoList = - backendSerializationProxy.getNamedStateSerializationProxies(); + List> restoredMetaInfoSnapshots = + backendSerializationProxy.getStateMetaInfoSnapshots(); // Recreate all PartitionableListStates from the meta info - for (OperatorBackendSerializationProxy.StateMetaInfo stateMetaInfo : metaInfoList) { - PartitionableListState listState = registeredStates.get(stateMetaInfo.getName()); + for (RegisteredOperatorBackendStateMetaInfo.Snapshot restoredMetaInfo : restoredMetaInfoSnapshots) { + PartitionableListState listState = registeredStates.get(restoredMetaInfo.getName()); if (null == listState) { listState = new PartitionableListState<>( - stateMetaInfo.getName(), - stateMetaInfo.getStateSerializer(), - stateMetaInfo.getMode()); + new RegisteredOperatorBackendStateMetaInfo<>( + restoredMetaInfo.getName(), + restoredMetaInfo.getPartitionStateSerializer(), + restoredMetaInfo.getAssignmentMode())); - registeredStates.put(listState.getName(), listState); + registeredStates.put(listState.getStateMetaInfo().getName(), listState); } else { - Preconditions.checkState(listState.getPartitionStateSerializer().canRestoreFrom( - stateMetaInfo.getStateSerializer()), "Incompatible state serializers found: " + - listState.getPartitionStateSerializer() + " is not compatible with " + - stateMetaInfo.getStateSerializer()); + // TODO with eager state registration in place, check here for serializer migration strategies } } @@ -341,7 +328,6 @@ public void restore(Collection restoreSnapshots) throws Exc } /** - * * Implementation of operator list state. * * @param the type of an operator state partition. @@ -349,19 +335,9 @@ public void restore(Collection restoreSnapshots) throws Exc static final class PartitionableListState implements ListState { /** - * The name of the state, as registered by the user - */ - private final String name; - - /** - * The type serializer for the elements in the state list - */ - private final TypeSerializer partitionStateSerializer; - - /** - * The mode how elements in this state are assigned to tasks during restore + * Meta information of the state, including state name, assignment mode, and serializer */ - private final OperatorStateHandle.Mode assignmentMode; + private final RegisteredOperatorBackendStateMetaInfo stateMetaInfo; /** * The internal list the holds the elements of the state @@ -373,46 +349,26 @@ static final class PartitionableListState implements ListState { */ private final ArrayListSerializer internalListCopySerializer; - public PartitionableListState( - String name, - TypeSerializer partitionStateSerializer, - OperatorStateHandle.Mode assignmentMode) { - - this(name, partitionStateSerializer, assignmentMode, new ArrayList()); + public PartitionableListState(RegisteredOperatorBackendStateMetaInfo stateMetaInfo) { + this(stateMetaInfo, new ArrayList()); } private PartitionableListState( - String name, - TypeSerializer partitionStateSerializer, - OperatorStateHandle.Mode assignmentMode, + RegisteredOperatorBackendStateMetaInfo stateMetaInfo, ArrayList internalList) { - this.name = Preconditions.checkNotNull(name); - this.partitionStateSerializer = Preconditions.checkNotNull(partitionStateSerializer); - this.assignmentMode = Preconditions.checkNotNull(assignmentMode); + this.stateMetaInfo = Preconditions.checkNotNull(stateMetaInfo); this.internalList = Preconditions.checkNotNull(internalList); - this.internalListCopySerializer = new ArrayListSerializer<>(partitionStateSerializer); + this.internalListCopySerializer = new ArrayListSerializer<>(stateMetaInfo.getPartitionStateSerializer()); } private PartitionableListState(PartitionableListState toCopy) { - this( - toCopy.name, - toCopy.partitionStateSerializer.duplicate(), - toCopy.assignmentMode, - toCopy.internalListCopySerializer.copy(toCopy.internalList)); - } - - public String getName() { - return name; - } - - public OperatorStateHandle.Mode getAssignmentMode() { - return assignmentMode; + this(toCopy.stateMetaInfo, toCopy.internalListCopySerializer.copy(toCopy.internalList)); } - public TypeSerializer getPartitionStateSerializer() { - return partitionStateSerializer; + public RegisteredOperatorBackendStateMetaInfo getStateMetaInfo() { + return stateMetaInfo; } public List getInternalList() { @@ -441,8 +397,7 @@ public void add(S value) { @Override public String toString() { return "PartitionableListState{" + - "name='" + name + '\'' + - ", assignmentMode=" + assignmentMode + + "stateMetaInfo=" + stateMetaInfo + ", internalList=" + internalList + '}'; } @@ -456,7 +411,7 @@ public long[] write(FSDataOutputStream out) throws IOException { for (int i = 0; i < internalList.size(); ++i) { S element = internalList.get(i); partitionOffsets[i] = out.getPos(); - partitionStateSerializer.serialize(element, dov); + getStateMetaInfo().getPartitionStateSerializer().serialize(element, dov); } return partitionOffsets; @@ -466,7 +421,6 @@ public long[] write(FSDataOutputStream out) throws IOException { private ListState getListState( ListStateDescriptor stateDescriptor, OperatorStateHandle.Mode mode) throws IOException { - Preconditions.checkNotNull(stateDescriptor); stateDescriptor.initializeSerializerUnlessSet(getExecutionConfig()); @@ -478,23 +432,27 @@ private ListState getListState( PartitionableListState partitionableListState = (PartitionableListState) registeredStates.get(name); if (null == partitionableListState) { - partitionableListState = new PartitionableListState<>( - name, - partitionStateSerializer, - mode); + new RegisteredOperatorBackendStateMetaInfo<>( + name, + partitionStateSerializer, + mode)); registeredStates.put(name, partitionableListState); } else { + // TODO with eager registration in place, these checks should be moved to restore() + Preconditions.checkState( - partitionableListState.getAssignmentMode().equals(mode), - "Incompatible assignment mode. Provided: " + mode + ", expected: " + - partitionableListState.getAssignmentMode()); + partitionableListState.getStateMetaInfo().getName().equals(name), + "Incompatible state names. " + + "Was [" + partitionableListState.getStateMetaInfo().getName() + "], " + + "registered with [" + name + "]."); + Preconditions.checkState( - stateDescriptor.getElementSerializer(). - canRestoreFrom(partitionableListState.getPartitionStateSerializer()), - "Incompatible type serializers. Provided: " + stateDescriptor.getElementSerializer() + - ", found: " + partitionableListState.getPartitionStateSerializer()); + partitionableListState.getStateMetaInfo().getAssignmentMode().equals(mode), + "Incompatible state assignment modes. " + + "Was [" + partitionableListState.getStateMetaInfo().getAssignmentMode() + "], " + + "registered with [" + mode + "]."); } return partitionableListState; @@ -509,7 +467,7 @@ private static void deserializeStateValues( long[] offsets = metaInfo.getOffsets(); if (null != offsets) { DataInputView div = new DataInputViewStreamWrapper(in); - TypeSerializer serializer = stateListForName.getPartitionStateSerializer(); + TypeSerializer serializer = stateListForName.getStateMetaInfo().getPartitionStateSerializer(); for (long offset : offsets) { in.seek(offset); stateListForName.add(serializer.deserialize(div)); 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 5661c38d4a6fc..467c1aec2380a 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,10 +18,8 @@ package org.apache.flink.runtime.state; -import org.apache.flink.api.common.state.StateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSerializationProxy; -import org.apache.flink.core.io.IOReadableWritable; import org.apache.flink.core.io.VersionMismatchException; import org.apache.flink.core.io.VersionedIOReadableWritable; import org.apache.flink.core.memory.DataInputView; @@ -38,10 +36,10 @@ */ public class KeyedBackendSerializationProxy extends VersionedIOReadableWritable { - public static final int VERSION = 2; + public static final int VERSION = 3; - private TypeSerializerSerializationProxy keySerializerProxy; - private List> namedStateSerializationProxies; + private TypeSerializer keySerializer; + private List> stateMetaInfoSnapshots; private int restoredVersion; private ClassLoader userCodeClassLoader; @@ -50,19 +48,25 @@ public KeyedBackendSerializationProxy(ClassLoader userCodeClassLoader) { this.userCodeClassLoader = Preconditions.checkNotNull(userCodeClassLoader); } - public KeyedBackendSerializationProxy(TypeSerializer keySerializer, List> namedStateSerializationProxies) { - this.keySerializerProxy = new TypeSerializerSerializationProxy<>(Preconditions.checkNotNull(keySerializer)); - this.namedStateSerializationProxies = Preconditions.checkNotNull(namedStateSerializationProxies); + public KeyedBackendSerializationProxy( + TypeSerializer keySerializer, + List> stateMetaInfoSnapshots) { + + this.keySerializer = Preconditions.checkNotNull(keySerializer); + + Preconditions.checkNotNull(stateMetaInfoSnapshots); + Preconditions.checkArgument(stateMetaInfoSnapshots.size() <= Short.MAX_VALUE); + this.stateMetaInfoSnapshots = stateMetaInfoSnapshots; + this.restoredVersion = VERSION; - Preconditions.checkArgument(namedStateSerializationProxies.size() <= Short.MAX_VALUE); } - public List> getNamedStateSerializationProxies() { - return namedStateSerializationProxies; + public List> getStateMetaInfoSnapshots() { + return stateMetaInfoSnapshots; } - public TypeSerializerSerializationProxy getKeySerializerProxy() { - return keySerializerProxy; + public TypeSerializer getKeySerializer() { + return keySerializer; } @Override @@ -82,20 +86,22 @@ protected void resolveVersionRead(int foundVersion) throws VersionMismatchExcept @Override public boolean isCompatibleVersion(int version) { - // we are compatible with version 2 (Flink 1.3.x) and version 1 (Flink 1.2.x) - return super.isCompatibleVersion(version) || version == 1; + // we are compatible with version 3 (Flink 1.3.x) and version 1 & 2 (Flink 1.2.x) + return super.isCompatibleVersion(version) || version == 2 || version == 1; } @Override public void write(DataOutputView out) throws IOException { super.write(out); - keySerializerProxy.write(out); + new TypeSerializerSerializationProxy<>(keySerializer).write(out); - out.writeShort(namedStateSerializationProxies.size()); + out.writeShort(stateMetaInfoSnapshots.size()); - for (StateMetaInfo kvState : namedStateSerializationProxies) { - kvState.write(out); + for (RegisteredKeyedBackendStateMetaInfo.Snapshot metaInfo : stateMetaInfoSnapshots) { + KeyedBackendStateMetaInfoSnapshotReaderWriters + .getWriterForVersion(VERSION, metaInfo) + .writeStateMetaInfo(out); } } @@ -103,132 +109,18 @@ public void write(DataOutputView out) throws IOException { public void read(DataInputView in) throws IOException { super.read(in); - keySerializerProxy = new TypeSerializerSerializationProxy<>(userCodeClassLoader); + final TypeSerializerSerializationProxy keySerializerProxy = + new TypeSerializerSerializationProxy<>(userCodeClassLoader); keySerializerProxy.read(in); + this.keySerializer = keySerializerProxy.getTypeSerializer(); int numKvStates = in.readShort(); - namedStateSerializationProxies = new ArrayList<>(numKvStates); - for (int i = 0; i < numKvStates; ++i) { - StateMetaInfo stateSerializationProxy = new StateMetaInfo<>(userCodeClassLoader); - stateSerializationProxy.read(in); - namedStateSerializationProxies.add(stateSerializationProxy); - } - } - - //---------------------------------------------------------------------------------------------------------------------- - - /** - * This is the serialization proxy for {@link RegisteredBackendStateMetaInfo} for a single registered state in a - * keyed backend. - */ - public static class StateMetaInfo implements IOReadableWritable { - - private StateDescriptor.Type stateType; - private String stateName; - private TypeSerializerSerializationProxy namespaceSerializerSerializationProxy; - private TypeSerializerSerializationProxy stateSerializerSerializationProxy; - - private ClassLoader userClassLoader; - - StateMetaInfo(ClassLoader userClassLoader) { - this.userClassLoader = Preconditions.checkNotNull(userClassLoader); - } - - public StateMetaInfo( - StateDescriptor.Type stateType, - String name, - TypeSerializer namespaceSerializer, - TypeSerializer stateSerializer) { - - this.stateType = Preconditions.checkNotNull(stateType); - this.stateName = Preconditions.checkNotNull(name); - this.namespaceSerializerSerializationProxy = new TypeSerializerSerializationProxy<>(Preconditions.checkNotNull(namespaceSerializer)); - this.stateSerializerSerializationProxy = new TypeSerializerSerializationProxy<>(Preconditions.checkNotNull(stateSerializer)); - } - - public StateDescriptor.Type getStateType() { - return stateType; - } - - public void setStateType(StateDescriptor.Type stateType) { - this.stateType = stateType; - } - - public String getStateName() { - return stateName; - } - - public void setStateName(String stateName) { - this.stateName = stateName; - } - - public TypeSerializerSerializationProxy getNamespaceSerializerSerializationProxy() { - return namespaceSerializerSerializationProxy; - } - - public void setNamespaceSerializerSerializationProxy(TypeSerializerSerializationProxy namespaceSerializerSerializationProxy) { - this.namespaceSerializerSerializationProxy = namespaceSerializerSerializationProxy; - } - - public TypeSerializerSerializationProxy getStateSerializerSerializationProxy() { - return stateSerializerSerializationProxy; - } - - public void setStateSerializerSerializationProxy(TypeSerializerSerializationProxy stateSerializerSerializationProxy) { - this.stateSerializerSerializationProxy = stateSerializerSerializationProxy; - } - - @Override - public void write(DataOutputView out) throws IOException { - out.writeInt(getStateType().ordinal()); - out.writeUTF(getStateName()); - - getNamespaceSerializerSerializationProxy().write(out); - getStateSerializerSerializationProxy().write(out); - } - - @Override - public void read(DataInputView in) throws IOException { - int enumOrdinal = in.readInt(); - setStateType(StateDescriptor.Type.values()[enumOrdinal]); - setStateName(in.readUTF()); - - namespaceSerializerSerializationProxy = new TypeSerializerSerializationProxy<>(userClassLoader); - namespaceSerializerSerializationProxy.read(in); - - stateSerializerSerializationProxy = new TypeSerializerSerializationProxy<>(userClassLoader); - stateSerializerSerializationProxy.read(in); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - StateMetaInfo that = (StateMetaInfo) o; - - if (!getStateName().equals(that.getStateName())) { - return false; - } - - if (!getNamespaceSerializerSerializationProxy().equals(that.getNamespaceSerializerSerializationProxy())) { - return false; - } - - return getStateSerializerSerializationProxy().equals(that.getStateSerializerSerializationProxy()); - } - - @Override - public int hashCode() { - int result = getStateName().hashCode(); - result = 31 * result + getNamespaceSerializerSerializationProxy().hashCode(); - result = 31 * result + getStateSerializerSerializationProxy().hashCode(); - return result; + stateMetaInfoSnapshots = new ArrayList<>(numKvStates); + for (int i = 0; i < numKvStates; i++) { + stateMetaInfoSnapshots.add( + KeyedBackendStateMetaInfoSnapshotReaderWriters + .getReaderForVersion(restoredVersion, userCodeClassLoader) + .readStateMetaInfo(in)); } } } 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 new file mode 100644 index 0000000000000..83aa33576aded --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/KeyedBackendStateMetaInfoSnapshotReaderWriters.java @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.state; + +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializerSerializationProxy; +import org.apache.flink.api.common.typeutils.TypeSerializerUtil; +import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; +import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; + +/** + * Readers and writers for different versions of the {@link RegisteredKeyedBackendStateMetaInfo.Snapshot}. + * Outdated formats are also kept here for documentation of history backlog. + */ +public class KeyedBackendStateMetaInfoSnapshotReaderWriters { + + // ------------------------------------------------------------------------------- + // Writers + // - v1: Flink 1.2.x + // - v2: Flink 1.3.x + // ------------------------------------------------------------------------------- + + public static KeyedBackendStateMetaInfoWriter getWriterForVersion( + int version, RegisteredKeyedBackendStateMetaInfo.Snapshot stateMetaInfo) { + + switch (version) { + case 1: + case 2: + return new KeyedBackendStateMetaInfoWriterV1V2<>(stateMetaInfo); + + // current version + case KeyedBackendSerializationProxy.VERSION: + return new KeyedBackendStateMetaInfoWriterV3<>(stateMetaInfo); + + default: + // guard for future + throw new IllegalStateException( + "Unrecognized keyed backend state meta info writer version: " + version); + } + } + + public interface KeyedBackendStateMetaInfoWriter { + void writeStateMetaInfo(DataOutputView out) throws IOException; + } + + static abstract class AbstractKeyedBackendStateMetaInfoWriter implements KeyedBackendStateMetaInfoWriter { + + protected final RegisteredKeyedBackendStateMetaInfo.Snapshot stateMetaInfo; + + public AbstractKeyedBackendStateMetaInfoWriter(RegisteredKeyedBackendStateMetaInfo.Snapshot stateMetaInfo) { + this.stateMetaInfo = Preconditions.checkNotNull(stateMetaInfo); + } + + } + + static class KeyedBackendStateMetaInfoWriterV1V2 extends AbstractKeyedBackendStateMetaInfoWriter { + + public KeyedBackendStateMetaInfoWriterV1V2(RegisteredKeyedBackendStateMetaInfo.Snapshot stateMetaInfo) { + super(stateMetaInfo); + } + + @Override + public void writeStateMetaInfo(DataOutputView out) throws IOException { + out.writeInt(stateMetaInfo.getStateType().ordinal()); + out.writeUTF(stateMetaInfo.getName()); + + new TypeSerializerSerializationProxy<>(stateMetaInfo.getNamespaceSerializer()).write(out); + new TypeSerializerSerializationProxy<>(stateMetaInfo.getStateSerializer()).write(out); + } + } + + static class KeyedBackendStateMetaInfoWriterV3 extends AbstractKeyedBackendStateMetaInfoWriter { + + public KeyedBackendStateMetaInfoWriterV3(RegisteredKeyedBackendStateMetaInfo.Snapshot stateMetaInfo) { + super(stateMetaInfo); + } + + @Override + public void writeStateMetaInfo(DataOutputView out) throws IOException { + out.writeInt(stateMetaInfo.getStateType().ordinal()); + out.writeUTF(stateMetaInfo.getName()); + + // write in a way that allows us to be fault-tolerant and skip blocks in the case of java serialization failures + try ( + ByteArrayOutputStreamWithPos outWithPos = new ByteArrayOutputStreamWithPos(); + DataOutputViewStreamWrapper outViewWrapper = new DataOutputViewStreamWrapper(outWithPos)) { + + new TypeSerializerSerializationProxy<>(stateMetaInfo.getNamespaceSerializer()).write(outViewWrapper); + + // write current offset, which represents the start offset of the state serializer + out.writeInt(outWithPos.getPosition()); + new TypeSerializerSerializationProxy<>(stateMetaInfo.getStateSerializer()).write(outViewWrapper); + + // write current offset, which represents the start of the configuration snapshots + out.writeInt(outWithPos.getPosition()); + TypeSerializerUtil.writeSerializerConfigSnapshot(outViewWrapper, stateMetaInfo.getNamespaceSerializerConfigSnapshot()); + TypeSerializerUtil.writeSerializerConfigSnapshot(outViewWrapper, stateMetaInfo.getStateSerializerConfigSnapshot()); + + // write total number of bytes and then flush + out.writeInt(outWithPos.getPosition()); + out.write(outWithPos.getBuf(), 0, outWithPos.getPosition()); + } + } + } + + + // ------------------------------------------------------------------------------- + // Readers + // - v1: Flink 1.2.x + // - v2: Flink 1.3.x + // ------------------------------------------------------------------------------- + + public static KeyedBackendStateMetaInfoReader getReaderForVersion( + int version, ClassLoader userCodeClassLoader) { + + switch (version) { + case 1: + case 2: + return new KeyedBackendStateMetaInfoReaderV1V2<>(userCodeClassLoader); + + // current version + case KeyedBackendSerializationProxy.VERSION: + return new KeyedBackendStateMetaInfoReaderV3<>(userCodeClassLoader); + + default: + // guard for future + throw new IllegalStateException( + "Unrecognized keyed backend state meta info reader version: " + version); + } + } + + public interface KeyedBackendStateMetaInfoReader { + RegisteredKeyedBackendStateMetaInfo.Snapshot readStateMetaInfo(DataInputView in) throws IOException; + } + + static abstract class AbstractKeyedBackendStateMetaInfoReader implements KeyedBackendStateMetaInfoReader { + + protected final ClassLoader userCodeClassLoader; + + public AbstractKeyedBackendStateMetaInfoReader(ClassLoader userCodeClassLoader) { + this.userCodeClassLoader = Preconditions.checkNotNull(userCodeClassLoader); + } + + } + + static class KeyedBackendStateMetaInfoReaderV1V2 extends AbstractKeyedBackendStateMetaInfoReader { + + public KeyedBackendStateMetaInfoReaderV1V2(ClassLoader userCodeClassLoader) { + super(userCodeClassLoader); + } + + @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()); + + final TypeSerializerSerializationProxy namespaceSerializerProxy = + new TypeSerializerSerializationProxy<>(userCodeClassLoader); + namespaceSerializerProxy.read(in); + metaInfo.setNamespaceSerializer(namespaceSerializerProxy.getTypeSerializer()); + + final TypeSerializerSerializationProxy stateSerializerProxy = + new TypeSerializerSerializationProxy<>(userCodeClassLoader); + stateSerializerProxy.read(in); + metaInfo.setStateSerializer(stateSerializerProxy.getTypeSerializer()); + + // older versions do not contain the configuration snapshot + metaInfo.setNamespaceSerializerConfigSnapshot(null); + metaInfo.setStateSerializerConfigSnapshot(null); + + return metaInfo; + } + } + + static class KeyedBackendStateMetaInfoReaderV3 extends AbstractKeyedBackendStateMetaInfoReader { + + public KeyedBackendStateMetaInfoReaderV3(ClassLoader userCodeClassLoader) { + super(userCodeClassLoader); + } + + @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()); + + // read offsets + int stateSerializerStartOffset = in.readInt(); + int configSnapshotsStartOffset = in.readInt(); + + int totalBytes = in.readInt(); + + byte[] buffer = new byte[totalBytes]; + in.readFully(buffer); + + ByteArrayInputStreamWithPos inWithPos = new ByteArrayInputStreamWithPos(buffer); + DataInputViewStreamWrapper inViewWrapper = new DataInputViewStreamWrapper(inWithPos); + + try { + final TypeSerializerSerializationProxy namespaceSerializerProxy = + new TypeSerializerSerializationProxy<>(userCodeClassLoader); + namespaceSerializerProxy.read(inViewWrapper); + metaInfo.setNamespaceSerializer(namespaceSerializerProxy.getTypeSerializer()); + } catch (IOException e) { + metaInfo.setNamespaceSerializer(null); + } + + // make sure we start from the state serializer bytes position + inWithPos.setPosition(stateSerializerStartOffset); + try { + final TypeSerializerSerializationProxy stateSerializerProxy = + new TypeSerializerSerializationProxy<>(userCodeClassLoader); + stateSerializerProxy.read(inViewWrapper); + metaInfo.setStateSerializer(stateSerializerProxy.getTypeSerializer()); + } catch (IOException e) { + metaInfo.setStateSerializer(null); + } + + // make sure we start from the config snapshot bytes position + inWithPos.setPosition(configSnapshotsStartOffset); + metaInfo.setNamespaceSerializerConfigSnapshot( + TypeSerializerUtil.readSerializerConfigSnapshot(inViewWrapper, userCodeClassLoader)); + metaInfo.setStateSerializerConfigSnapshot( + TypeSerializerUtil.readSerializerConfigSnapshot(inViewWrapper, 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 d571dccc8fc28..476aaac931546 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 @@ -18,15 +18,10 @@ package org.apache.flink.runtime.state; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream; -import org.apache.flink.api.java.typeutils.runtime.DataOutputViewStream; -import org.apache.flink.core.io.IOReadableWritable; +import org.apache.flink.core.io.VersionMismatchException; 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; @@ -39,18 +34,24 @@ */ public class OperatorBackendSerializationProxy extends VersionedIOReadableWritable { - private static final int VERSION = 1; + public static final int VERSION = 2; - private List> namedStateSerializationProxies; + private List> stateMetaInfoSnapshots; private ClassLoader userCodeClassLoader; + private int restoredVersion; + public OperatorBackendSerializationProxy(ClassLoader userCodeClassLoader) { this.userCodeClassLoader = Preconditions.checkNotNull(userCodeClassLoader); } - public OperatorBackendSerializationProxy(List> namedStateSerializationProxies) { - this.namedStateSerializationProxies = Preconditions.checkNotNull(namedStateSerializationProxies); - Preconditions.checkArgument(namedStateSerializationProxies.size() <= Short.MAX_VALUE); + public OperatorBackendSerializationProxy( + List> stateMetaInfoSnapshots) { + + this.stateMetaInfoSnapshots = Preconditions.checkNotNull(stateMetaInfoSnapshots); + Preconditions.checkArgument(stateMetaInfoSnapshots.size() <= Short.MAX_VALUE); + + this.restoredVersion = VERSION; } @Override @@ -59,129 +60,44 @@ public int getVersion() { } @Override - public void write(DataOutputView out) throws IOException { - super.write(out); - - out.writeShort(namedStateSerializationProxies.size()); - - for (StateMetaInfo kvState : namedStateSerializationProxies) { - kvState.write(out); - } + protected void resolveVersionRead(int foundVersion) throws VersionMismatchException { + super.resolveVersionRead(foundVersion); + this.restoredVersion = foundVersion; } @Override - public void read(DataInputView out) throws IOException { - super.read(out); - - int numKvStates = out.readShort(); - namedStateSerializationProxies = new ArrayList<>(numKvStates); - for (int i = 0; i < numKvStates; ++i) { - StateMetaInfo stateSerializationProxy = new StateMetaInfo<>(userCodeClassLoader); - stateSerializationProxy.read(out); - namedStateSerializationProxies.add(stateSerializationProxy); - } + public boolean isCompatibleVersion(int version) { + // we are compatible with version 2 (Flink 1.3.x) and version 1 (Flink 1.2.x) + return super.isCompatibleVersion(version) || version == 1; } - public List> getNamedStateSerializationProxies() { - return namedStateSerializationProxies; - } - - //---------------------------------------------------------------------------------------------------------------------- - - public static class StateMetaInfo implements IOReadableWritable { - - private String name; - private TypeSerializer stateSerializer; - private OperatorStateHandle.Mode mode; - - private ClassLoader userClassLoader; - - @VisibleForTesting - public StateMetaInfo(ClassLoader userClassLoader) { - this.userClassLoader = Preconditions.checkNotNull(userClassLoader); - } - - public StateMetaInfo(String name, TypeSerializer stateSerializer, OperatorStateHandle.Mode mode) { - this.name = Preconditions.checkNotNull(name); - this.stateSerializer = Preconditions.checkNotNull(stateSerializer); - this.mode = Preconditions.checkNotNull(mode); - } - - public String getName() { - return name; - } - - public void setName(String name) { - this.name = name; - } - - public TypeSerializer getStateSerializer() { - return stateSerializer; - } - - public void setStateSerializer(TypeSerializer stateSerializer) { - this.stateSerializer = stateSerializer; - } - - public OperatorStateHandle.Mode getMode() { - return mode; - } - - public void setMode(OperatorStateHandle.Mode mode) { - this.mode = mode; - } - - @Override - public void write(DataOutputView out) throws IOException { - out.writeUTF(getName()); - out.writeByte(getMode().ordinal()); - DataOutputViewStream dos = new DataOutputViewStream(out); - InstantiationUtil.serializeObject(dos, getStateSerializer()); - } + @Override + public void write(DataOutputView out) throws IOException { + super.write(out); - @Override - public void read(DataInputView in) throws IOException { - setName(in.readUTF()); - setMode(OperatorStateHandle.Mode.values()[in.readByte()]); - DataInputViewStream dis = new DataInputViewStream(in); - try { - TypeSerializer stateSerializer = InstantiationUtil.deserializeObject(dis, userClassLoader); - setStateSerializer(stateSerializer); - } catch (ClassNotFoundException exception) { - throw new IOException(exception); - } + out.writeShort(stateMetaInfoSnapshots.size()); + for (RegisteredOperatorBackendStateMetaInfo.Snapshot kvState : stateMetaInfoSnapshots) { + OperatorBackendStateMetaInfoSnapshotReaderWriters + .getWriterForVersion(VERSION, kvState) + .writeStateMetaInfo(out); } + } - @Override - public boolean equals(Object o) { - - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - StateMetaInfo metaInfo = (StateMetaInfo) o; - - if (!getName().equals(metaInfo.getName())) { - return false; - } - - if (!getStateSerializer().equals(metaInfo.getStateSerializer())) { - return false; - } - - return getMode() == metaInfo.getMode(); + @Override + public void read(DataInputView in) throws IOException { + super.read(in); + + int numKvStates = in.readShort(); + stateMetaInfoSnapshots = new ArrayList<>(numKvStates); + for (int i = 0; i < numKvStates; i++) { + stateMetaInfoSnapshots.add( + OperatorBackendStateMetaInfoSnapshotReaderWriters + .getReaderForVersion(restoredVersion, userCodeClassLoader) + .readStateMetaInfo(in)); } + } - @Override - public int hashCode() { - int result = getName().hashCode(); - result = 31 * result + getStateSerializer().hashCode(); - result = 31 * result + getMode().hashCode(); - return result; - } + public List> getStateMetaInfoSnapshots() { + return stateMetaInfoSnapshots; } } 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 new file mode 100644 index 0000000000000..9ab106bc492db --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/OperatorBackendStateMetaInfoSnapshotReaderWriters.java @@ -0,0 +1,233 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.state; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSerializationProxy; +import org.apache.flink.api.common.typeutils.TypeSerializerUtil; +import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream; +import org.apache.flink.core.memory.ByteArrayInputStreamWithPos; +import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.InstantiationUtil; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; + +/** + * Readers and writers for different versions of the {@link RegisteredOperatorBackendStateMetaInfo.Snapshot}. + * Outdated formats are also kept here for documentation of history backlog. + */ +public class OperatorBackendStateMetaInfoSnapshotReaderWriters { + + // ------------------------------------------------------------------------------- + // Writers + // - v1: Flink 1.2.x + // - v2: Flink 1.3.x + // ------------------------------------------------------------------------------- + + public static OperatorBackendStateMetaInfoWriter getWriterForVersion( + int version, RegisteredOperatorBackendStateMetaInfo.Snapshot stateMetaInfo) { + + switch (version) { + case 1: + return new OperatorBackendStateMetaInfoWriterV1<>(stateMetaInfo); + + // current version + case OperatorBackendSerializationProxy.VERSION: + return new OperatorBackendStateMetaInfoWriterV2<>(stateMetaInfo); + + default: + // guard for future + throw new IllegalStateException( + "Unrecognized operator backend state meta info writer version: " + version); + } + } + + public interface OperatorBackendStateMetaInfoWriter { + void writeStateMetaInfo(DataOutputView out) throws IOException; + } + + public static abstract class AbstractOperatorBackendStateMetaInfoWriter + implements OperatorBackendStateMetaInfoWriter { + + protected final RegisteredOperatorBackendStateMetaInfo.Snapshot stateMetaInfo; + + public AbstractOperatorBackendStateMetaInfoWriter(RegisteredOperatorBackendStateMetaInfo.Snapshot stateMetaInfo) { + this.stateMetaInfo = Preconditions.checkNotNull(stateMetaInfo); + } + } + + public static class OperatorBackendStateMetaInfoWriterV1 extends AbstractOperatorBackendStateMetaInfoWriter { + + public OperatorBackendStateMetaInfoWriterV1(RegisteredOperatorBackendStateMetaInfo.Snapshot stateMetaInfo) { + super(stateMetaInfo); + } + + @Override + public void writeStateMetaInfo(DataOutputView out) throws IOException { + out.writeUTF(stateMetaInfo.getName()); + out.writeByte(stateMetaInfo.getAssignmentMode().ordinal()); + new TypeSerializerSerializationProxy<>(stateMetaInfo.getPartitionStateSerializer()).write(out); + } + } + + public static class OperatorBackendStateMetaInfoWriterV2 extends AbstractOperatorBackendStateMetaInfoWriter { + + public OperatorBackendStateMetaInfoWriterV2(RegisteredOperatorBackendStateMetaInfo.Snapshot stateMetaInfo) { + super(stateMetaInfo); + } + + @Override + public void writeStateMetaInfo(DataOutputView out) throws IOException { + out.writeUTF(stateMetaInfo.getName()); + out.writeByte(stateMetaInfo.getAssignmentMode().ordinal()); + + // write in a way that allows us to be fault-tolerant and skip blocks in the case of java serialization failures + try ( + ByteArrayOutputStreamWithPos outWithPos = new ByteArrayOutputStreamWithPos(); + DataOutputViewStreamWrapper outViewWrapper = new DataOutputViewStreamWrapper(outWithPos)) { + + new TypeSerializerSerializationProxy<>(stateMetaInfo.getPartitionStateSerializer()).write(outViewWrapper); + + // write the start offset of the config snapshot + out.writeInt(outWithPos.getPosition()); + TypeSerializerUtil.writeSerializerConfigSnapshot( + outViewWrapper, + stateMetaInfo.getPartitionStateSerializerConfigSnapshot()); + + // write the total number of bytes and flush + out.writeInt(outWithPos.getPosition()); + out.write(outWithPos.getBuf(), 0, outWithPos.getPosition()); + } + } + } + + // ------------------------------------------------------------------------------- + // Readers + // - v1: Flink 1.2.x + // - v2: Flink 1.3.x + // ------------------------------------------------------------------------------- + + public static OperatorBackendStateMetaInfoReader getReaderForVersion( + int version, ClassLoader userCodeClassLoader) { + + switch (version) { + case 1: + return new OperatorBackendStateMetaInfoReaderV1<>(userCodeClassLoader); + + // current version + case OperatorBackendSerializationProxy.VERSION: + return new OperatorBackendStateMetaInfoReaderV2<>(userCodeClassLoader); + + default: + // guard for future + throw new IllegalStateException( + "Unrecognized operator backend state meta info reader version: " + version); + } + } + + public interface OperatorBackendStateMetaInfoReader { + RegisteredOperatorBackendStateMetaInfo.Snapshot readStateMetaInfo(DataInputView in) throws IOException; + } + + public static abstract class AbstractOperatorBackendStateMetaInfoReader + implements OperatorBackendStateMetaInfoReader { + + protected final ClassLoader userCodeClassLoader; + + public AbstractOperatorBackendStateMetaInfoReader(ClassLoader userCodeClassLoader) { + this.userCodeClassLoader = Preconditions.checkNotNull(userCodeClassLoader); + } + } + + public static class OperatorBackendStateMetaInfoReaderV1 extends AbstractOperatorBackendStateMetaInfoReader { + + public OperatorBackendStateMetaInfoReaderV1(ClassLoader userCodeClassLoader) { + super(userCodeClassLoader); + } + + @Override + public RegisteredOperatorBackendStateMetaInfo.Snapshot readStateMetaInfo(DataInputView in) throws IOException { + RegisteredOperatorBackendStateMetaInfo.Snapshot stateMetaInfo = + new RegisteredOperatorBackendStateMetaInfo.Snapshot<>(); + + stateMetaInfo.setName(in.readUTF()); + stateMetaInfo.setAssignmentMode(OperatorStateHandle.Mode.values()[in.readByte()]); + DataInputViewStream dis = new DataInputViewStream(in); + try { + TypeSerializer stateSerializer = InstantiationUtil.deserializeObject(dis, userCodeClassLoader); + stateMetaInfo.setPartitionStateSerializer(stateSerializer); + } catch (ClassNotFoundException exception) { + throw new IOException(exception); + } + + // old versions do not contain the partition state serializer's configuration snapshot + stateMetaInfo.setPartitionStateSerializerConfigSnapshot(null); + + return stateMetaInfo; + } + } + + public static class OperatorBackendStateMetaInfoReaderV2 extends AbstractOperatorBackendStateMetaInfoReader { + + public OperatorBackendStateMetaInfoReaderV2(ClassLoader userCodeClassLoader) { + super(userCodeClassLoader); + } + + @Override + public RegisteredOperatorBackendStateMetaInfo.Snapshot readStateMetaInfo(DataInputView in) throws IOException { + RegisteredOperatorBackendStateMetaInfo.Snapshot stateMetaInfo = + new RegisteredOperatorBackendStateMetaInfo.Snapshot<>(); + + stateMetaInfo.setName(in.readUTF()); + stateMetaInfo.setAssignmentMode(OperatorStateHandle.Mode.values()[in.readByte()]); + + // read start offset of configuration snapshot + int configSnapshotStartOffset = in.readInt(); + + int totalBytes = in.readInt(); + + byte[] buffer = new byte[totalBytes]; + in.readFully(buffer); + + ByteArrayInputStreamWithPos inWithPos = new ByteArrayInputStreamWithPos(buffer); + DataInputViewStreamWrapper inViewWrapper = new DataInputViewStreamWrapper(inWithPos); + + try { + final TypeSerializerSerializationProxy partitionStateSerializerProxy = + new TypeSerializerSerializationProxy<>(userCodeClassLoader); + partitionStateSerializerProxy.read(inViewWrapper); + stateMetaInfo.setPartitionStateSerializer(partitionStateSerializerProxy.getTypeSerializer()); + } catch (IOException e) { + stateMetaInfo.setPartitionStateSerializer(null); + } + + // make sure we start from the partition state serializer bytes position + inWithPos.setPosition(configSnapshotStartOffset); + stateMetaInfo.setPartitionStateSerializerConfigSnapshot( + TypeSerializerUtil.readSerializerConfigSnapshot(inViewWrapper, userCodeClassLoader)); + + return stateMetaInfo; + } + } +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBackendStateMetaInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBackendStateMetaInfo.java deleted file mode 100644 index 0d4b3c82e1b90..0000000000000 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredBackendStateMetaInfo.java +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.runtime.state; - -import org.apache.flink.api.common.state.StateDescriptor; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.migration.MigrationNamespaceSerializerProxy; - -import static org.apache.flink.util.Preconditions.checkNotNull; - -/** - * Compound meta information for a registered state in a keyed state backend. This combines all serializers and the - * state name. - * - * @param Type of namespace - * @param Type of state value - */ -public class RegisteredBackendStateMetaInfo { - - private final StateDescriptor.Type stateType; - private final String name; - private final TypeSerializer namespaceSerializer; - private final TypeSerializer stateSerializer; - - public RegisteredBackendStateMetaInfo(KeyedBackendSerializationProxy.StateMetaInfo metaInfoProxy) { - this( - metaInfoProxy.getStateType(), - metaInfoProxy.getStateName(), - metaInfoProxy.getNamespaceSerializerSerializationProxy().getTypeSerializer(), - metaInfoProxy.getStateSerializerSerializationProxy().getTypeSerializer()); - } - - public RegisteredBackendStateMetaInfo( - StateDescriptor.Type stateType, - String name, - TypeSerializer namespaceSerializer, - TypeSerializer stateSerializer) { - - this.stateType = checkNotNull(stateType); - this.name = checkNotNull(name); - this.namespaceSerializer = checkNotNull(namespaceSerializer); - this.stateSerializer = checkNotNull(stateSerializer); - } - - public StateDescriptor.Type getStateType() { - return stateType; - } - - public String getName() { - return name; - } - - public TypeSerializer getNamespaceSerializer() { - return namespaceSerializer; - } - - public TypeSerializer getStateSerializer() { - return stateSerializer; - } - - public boolean canRestoreFrom(RegisteredBackendStateMetaInfo other) { - - if (this == other) { - return true; - } - - if (null == other) { - return false; - } - - if (!stateType.equals(StateDescriptor.Type.UNKNOWN) - && !other.stateType.equals(StateDescriptor.Type.UNKNOWN) - && !stateType.equals(other.stateType)) { - return false; - } - - if (!name.equals(other.getName())) { - return false; - } - - return (stateSerializer.canRestoreFrom(other.stateSerializer)) && - (namespaceSerializer.canRestoreFrom(other.namespaceSerializer) - // we also check if there is just a migration proxy that should be replaced by any real serializer - || other.namespaceSerializer instanceof MigrationNamespaceSerializerProxy); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - RegisteredBackendStateMetaInfo that = (RegisteredBackendStateMetaInfo) o; - - if (!stateType.equals(that.stateType)) { - return false; - } - - if (!getName().equals(that.getName())) { - return false; - } - - return getStateSerializer().equals(that.getStateSerializer()) - && getNamespaceSerializer().equals(that.getNamespaceSerializer()); - } - - @Override - public String toString() { - return "RegisteredBackendStateMetaInfo{" + - "stateType=" + stateType + - ", name='" + name + '\'' + - ", namespaceSerializer=" + namespaceSerializer + - ", stateSerializer=" + stateSerializer + - '}'; - } - - @Override - public int hashCode() { - 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); - 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 new file mode 100644 index 0000000000000..e1a7e067668ed --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredKeyedBackendStateMetaInfo.java @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.state; + +import org.apache.flink.api.common.state.StateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +import org.apache.flink.util.Preconditions; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Compound meta information for a registered state in a keyed state backend. This combines all serializers and the + * state name. + * + * @param Type of namespace + * @param Type of state value + */ +public class RegisteredKeyedBackendStateMetaInfo { + + private final StateDescriptor.Type stateType; + private final String name; + private final TypeSerializer namespaceSerializer; + private final TypeSerializer stateSerializer; + + public RegisteredKeyedBackendStateMetaInfo( + StateDescriptor.Type stateType, + String name, + TypeSerializer namespaceSerializer, + TypeSerializer stateSerializer) { + + this.stateType = checkNotNull(stateType); + this.name = checkNotNull(name); + this.namespaceSerializer = checkNotNull(namespaceSerializer); + this.stateSerializer = checkNotNull(stateSerializer); + } + + public StateDescriptor.Type getStateType() { + return stateType; + } + + public String getName() { + return name; + } + + public TypeSerializer getNamespaceSerializer() { + return namespaceSerializer; + } + + public TypeSerializer getStateSerializer() { + return stateSerializer; + } + + public Snapshot snapshot() { + return new Snapshot<>( + stateType, + name, + namespaceSerializer.duplicate(), + stateSerializer.duplicate(), + namespaceSerializer.snapshotConfiguration(), + stateSerializer.snapshotConfiguration()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + RegisteredKeyedBackendStateMetaInfo that = (RegisteredKeyedBackendStateMetaInfo) o; + + if (!stateType.equals(that.stateType)) { + return false; + } + + if (!getName().equals(that.getName())) { + return false; + } + + return getStateSerializer().equals(that.getStateSerializer()) + && getNamespaceSerializer().equals(that.getNamespaceSerializer()); + } + + @Override + public String toString() { + return "RegisteredKeyedBackendStateMetaInfo{" + + "stateType=" + stateType + + ", name='" + name + '\'' + + ", namespaceSerializer=" + namespaceSerializer + + ", stateSerializer=" + stateSerializer + + '}'; + } + + @Override + public int hashCode() { + int result = getName().hashCode(); + result = 31 * result + getStateType().hashCode(); + result = 31 * result + getNamespaceSerializer().hashCode(); + result = 31 * result + getStateSerializer().hashCode(); + return result; + } + + /** + * A consistent snapshot of a {@link RegisteredKeyedBackendStateMetaInfo}. + */ + public static class Snapshot { + + private StateDescriptor.Type stateType; + private String name; + private TypeSerializer namespaceSerializer; + private TypeSerializer stateSerializer; + private TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot; + private TypeSerializerConfigSnapshot stateSerializerConfigSnapshot; + + /** Empty constructor used when restoring the state meta info snapshot. */ + Snapshot() {} + + private Snapshot( + StateDescriptor.Type stateType, + String name, + TypeSerializer namespaceSerializer, + TypeSerializer stateSerializer, + TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot, + TypeSerializerConfigSnapshot stateSerializerConfigSnapshot) { + + 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); + } + + public StateDescriptor.Type getStateType() { + return stateType; + } + + void setStateType(StateDescriptor.Type stateType) { + this.stateType = stateType; + } + + public String getName() { + return name; + } + + 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() { + return namespaceSerializerConfigSnapshot; + } + + void setNamespaceSerializerConfigSnapshot(TypeSerializerConfigSnapshot namespaceSerializerConfigSnapshot) { + this.namespaceSerializerConfigSnapshot = namespaceSerializerConfigSnapshot; + } + + public TypeSerializerConfigSnapshot getStateSerializerConfigSnapshot() { + return stateSerializerConfigSnapshot; + } + + void setStateSerializerConfigSnapshot(TypeSerializerConfigSnapshot stateSerializerConfigSnapshot) { + this.stateSerializerConfigSnapshot = stateSerializerConfigSnapshot; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + Snapshot that = (Snapshot) o; + + if (!stateType.equals(that.stateType)) { + return false; + } + + if (!getName().equals(that.getName())) { + return false; + } + + // need to check for nulls because serializer and config snapshots may be null on restore + return + ((getStateSerializer() == null && that.getStateSerializer() == null) + || getStateSerializer().equals(that.getStateSerializer())) + && ((getNamespaceSerializer() == null && that.getNamespaceSerializer() == null) + || getNamespaceSerializer().equals(that.getNamespaceSerializer())) + && ((getNamespaceSerializerConfigSnapshot() == null && that.getNamespaceSerializerConfigSnapshot() == null) + || getNamespaceSerializerConfigSnapshot().equals(that.getNamespaceSerializerConfigSnapshot())) + && ((getStateSerializerConfigSnapshot() == null && that.getStateSerializerConfigSnapshot() == null) + || getStateSerializerConfigSnapshot().equals(that.getStateSerializerConfigSnapshot())); + } + + @Override + 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 new file mode 100644 index 0000000000000..b43fc9c537cc5 --- /dev/null +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/RegisteredOperatorBackendStateMetaInfo.java @@ -0,0 +1,198 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.runtime.state; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; +import org.apache.flink.util.Preconditions; + +/** + * Compound meta information for a registered state in an operator state backend. + * This contains the state name, assignment mode, and state partition serializer. + * + * @param Type of the state. + */ +public class RegisteredOperatorBackendStateMetaInfo { + + /** + * The name of the state, as registered by the user + */ + private final String name; + + /** + * The mode how elements in this state are assigned to tasks during restore + */ + private final OperatorStateHandle.Mode assignmentMode; + + /** + * The type serializer for the elements in the state list + */ + private final TypeSerializer partitionStateSerializer; + + public RegisteredOperatorBackendStateMetaInfo( + String name, + TypeSerializer partitionStateSerializer, + OperatorStateHandle.Mode assignmentMode) { + + this.name = Preconditions.checkNotNull(name); + this.partitionStateSerializer = Preconditions.checkNotNull(partitionStateSerializer); + this.assignmentMode = Preconditions.checkNotNull(assignmentMode); + } + + public String getName() { + return name; + } + + public OperatorStateHandle.Mode getAssignmentMode() { + return assignmentMode; + } + + public TypeSerializer getPartitionStateSerializer() { + return partitionStateSerializer; + } + + public Snapshot snapshot() { + return new Snapshot<>( + name, + assignmentMode, + partitionStateSerializer.duplicate(), + partitionStateSerializer.snapshotConfiguration()); + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + + if (obj == null) { + return false; + } + + return (obj instanceof RegisteredOperatorBackendStateMetaInfo) + && name.equals(((RegisteredOperatorBackendStateMetaInfo) obj).getName()) + && assignmentMode.equals(((RegisteredOperatorBackendStateMetaInfo) obj).getAssignmentMode()) + && partitionStateSerializer.equals(((RegisteredOperatorBackendStateMetaInfo) obj).getPartitionStateSerializer()); + } + + @Override + public int hashCode() { + int result = getName().hashCode(); + result = 31 * result + getAssignmentMode().hashCode(); + result = 31 * result + getPartitionStateSerializer().hashCode(); + return result; + } + + @Override + public String toString() { + return "RegisteredOperatorBackendStateMetaInfo{" + + "name='" + name + "\'" + + ", assignmentMode=" + assignmentMode + + ", partitionStateSerializer=" + partitionStateSerializer + + '}'; + } + + /** + * A consistent snapshot of a {@link RegisteredOperatorBackendStateMetaInfo}. + */ + 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. */ + Snapshot() {} + + private Snapshot( + String name, + OperatorStateHandle.Mode assignmentMode, + TypeSerializer partitionStateSerializer, + TypeSerializerConfigSnapshot partitionStateSerializerConfigSnapshot) { + + this.name = Preconditions.checkNotNull(name); + this.assignmentMode = Preconditions.checkNotNull(assignmentMode); + this.partitionStateSerializer = Preconditions.checkNotNull(partitionStateSerializer); + this.partitionStateSerializerConfigSnapshot = Preconditions.checkNotNull(partitionStateSerializerConfigSnapshot); + } + + public String getName() { + return name; + } + + void setName(String name) { + this.name = name; + } + + public OperatorStateHandle.Mode getAssignmentMode() { + return assignmentMode; + } + + 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; + } + + void setPartitionStateSerializerConfigSnapshot(TypeSerializerConfigSnapshot partitionStateSerializerConfigSnapshot) { + this.partitionStateSerializerConfigSnapshot = partitionStateSerializerConfigSnapshot; + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + + if (obj == null) { + return false; + } + + // need to check for nulls because serializer and config snapshots may be null on restore + return (obj instanceof Snapshot) + && name.equals(((Snapshot) obj).getName()) + && assignmentMode.equals(((Snapshot) obj).getAssignmentMode()) + && ((partitionStateSerializer == null && ((Snapshot) obj).getPartitionStateSerializer() == null) + || partitionStateSerializer.equals(((Snapshot) obj).getPartitionStateSerializer())) + && ((partitionStateSerializerConfigSnapshot == null && ((Snapshot) obj).getPartitionStateSerializerConfigSnapshot() == null) + || partitionStateSerializerConfigSnapshot.equals(((Snapshot) obj).getPartitionStateSerializerConfigSnapshot())); + } + + @Override + 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/VoidNamespaceSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/VoidNamespaceSerializer.java index 8b58891e36ba6..2800899280a21 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/VoidNamespaceSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/VoidNamespaceSerializer.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.migration.MigrationNamespaceSerializerProxy; import java.io.IOException; @@ -89,4 +90,11 @@ public void copy(DataInputView source, DataOutputView target) throws IOException public boolean canEqual(Object obj) { return obj instanceof VoidNamespaceSerializer; } + + @Override + protected boolean isCompatibleSerializationFormatIdentifier(String identifier) { + // we might be replacing a migration namespace serializer, in which case we just assume compatibility + return super.isCompatibleSerializationFormatIdentifier(identifier) + || identifier.equals(MigrationNamespaceSerializerProxy.class.getCanonicalName()); + } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTable.java index d63b6d3d06acf..7b61da10138ca 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTable.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo; +import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo; import org.apache.flink.runtime.state.StateTransformationFunction; import org.apache.flink.util.MathUtils; import org.apache.flink.util.Preconditions; @@ -196,7 +196,7 @@ public class CopyOnWriteStateTable extends StateTable implemen * @param keyContext the key context. * @param metaInfo the meta information, including the type serializer for state copy-on-write. */ - CopyOnWriteStateTable(InternalKeyContext keyContext, RegisteredBackendStateMetaInfo metaInfo) { + CopyOnWriteStateTable(InternalKeyContext keyContext, RegisteredKeyedBackendStateMetaInfo metaInfo) { this(keyContext, metaInfo, 1024); } @@ -209,7 +209,7 @@ public class CopyOnWriteStateTable extends StateTable implemen * @throws IllegalArgumentException when the capacity is less than zero. */ @SuppressWarnings("unchecked") - private CopyOnWriteStateTable(InternalKeyContext keyContext, RegisteredBackendStateMetaInfo metaInfo, int capacity) { + private CopyOnWriteStateTable(InternalKeyContext keyContext, RegisteredKeyedBackendStateMetaInfo metaInfo, int capacity) { super(keyContext, metaInfo); // initialized tables to EMPTY_TABLE. @@ -532,12 +532,12 @@ public TypeSerializer getNamespaceSerializer() { } @Override - public RegisteredBackendStateMetaInfo getMetaInfo() { + public RegisteredKeyedBackendStateMetaInfo getMetaInfo() { return metaInfo; } @Override - public void setMetaInfo(RegisteredBackendStateMetaInfo metaInfo) { + public void setMetaInfo(RegisteredKeyedBackendStateMetaInfo metaInfo) { this.metaInfo = metaInfo; } @@ -1063,4 +1063,4 @@ public void remove() { throw new UnsupportedOperationException("Read-only iterator"); } } -} \ No newline at end of file +} 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 aecc72e3bf869..866ed2830cd26 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 @@ -52,7 +52,7 @@ import org.apache.flink.runtime.state.KeyGroupsStateHandle; import org.apache.flink.runtime.state.KeyedBackendSerializationProxy; import org.apache.flink.runtime.state.KeyedStateHandle; -import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo; +import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo; import org.apache.flink.runtime.state.StreamStateHandle; import org.apache.flink.runtime.state.internal.InternalAggregatingState; import org.apache.flink.runtime.state.internal.InternalFoldingState; @@ -132,8 +132,8 @@ private StateTable tryRegisterStateTable( TypeSerializer namespaceSerializer, TypeSerializer valueSerializer) { - final RegisteredBackendStateMetaInfo newMetaInfo = - new RegisteredBackendStateMetaInfo<>(stateType, stateName, namespaceSerializer, valueSerializer); + final RegisteredKeyedBackendStateMetaInfo newMetaInfo = + new RegisteredKeyedBackendStateMetaInfo<>(stateType, stateName, namespaceSerializer, valueSerializer); @SuppressWarnings("unchecked") StateTable stateTable = (StateTable) stateTables.get(stateName); @@ -142,12 +142,27 @@ private StateTable tryRegisterStateTable( stateTable = newStateTable(newMetaInfo); stateTables.put(stateName, stateTable); } else { - if (!newMetaInfo.canRestoreFrom(stateTable.getMetaInfo())) { - throw new RuntimeException("Trying to access state using incompatible meta info, was " + - stateTable.getMetaInfo() + " trying access with " + newMetaInfo); + // TODO with eager registration in place, these checks should be moved to restorePartitionedState() + + Preconditions.checkState( + stateName.equals(stateTable.getMetaInfo().getName()), + "Incompatible state names. " + + "Was [" + stateTable.getMetaInfo().getName() + "], " + + "registered with [" + newMetaInfo.getName() + "]."); + + if (!newMetaInfo.getStateType().equals(StateDescriptor.Type.UNKNOWN) + && !stateTable.getMetaInfo().getStateType().equals(StateDescriptor.Type.UNKNOWN)) { + + Preconditions.checkState( + newMetaInfo.getStateType().equals(stateTable.getMetaInfo().getStateType()), + "Incompatible state types. " + + "Was [" + stateTable.getMetaInfo().getStateType() + "], " + + "registered with [" + newMetaInfo.getStateType() + "]."); } + stateTable.setMetaInfo(newMetaInfo); } + return stateTable; } @@ -240,21 +255,14 @@ public RunnableFuture snapshot( "Too many KV-States: " + stateTables.size() + ". Currently at most " + Short.MAX_VALUE + " states are supported"); - List> metaInfoProxyList = new ArrayList<>(stateTables.size()); + List> metaInfoSnapshots = new ArrayList<>(stateTables.size()); final Map kVStateToId = new HashMap<>(stateTables.size()); final Map, StateTableSnapshot> cowStateStableSnapshots = new HashedMap(stateTables.size()); for (Map.Entry> kvState : stateTables.entrySet()) { - RegisteredBackendStateMetaInfo metaInfo = kvState.getValue().getMetaInfo(); - KeyedBackendSerializationProxy.StateMetaInfo metaInfoProxy = new KeyedBackendSerializationProxy.StateMetaInfo( - metaInfo.getStateType(), - metaInfo.getName(), - metaInfo.getNamespaceSerializer(), - metaInfo.getStateSerializer()); - - metaInfoProxyList.add(metaInfoProxy); + metaInfoSnapshots.add(kvState.getValue().getMetaInfo().snapshot()); kVStateToId.put(kvState.getKey(), kVStateToId.size()); StateTable stateTable = kvState.getValue(); if (null != stateTable) { @@ -263,7 +271,7 @@ public RunnableFuture snapshot( } final KeyedBackendSerializationProxy serializationProxy = - new KeyedBackendSerializationProxy(keySerializer, metaInfoProxyList); + new KeyedBackendSerializationProxy(keySerializer, metaInfoSnapshots); //--------------------------------------------------- this becomes the end of sync part @@ -376,23 +384,29 @@ private void restorePartitionedState(Collection state) throws serializationProxy.read(inView); - List> metaInfoList = - serializationProxy.getNamedStateSerializationProxies(); + List> restoredMetaInfos = + serializationProxy.getStateMetaInfoSnapshots(); - for (KeyedBackendSerializationProxy.StateMetaInfo metaInfoSerializationProxy : metaInfoList) { + for (RegisteredKeyedBackendStateMetaInfo.Snapshot restoredMetaInfo : restoredMetaInfos) { - StateTable stateTable = stateTables.get(metaInfoSerializationProxy.getStateName()); + StateTable stateTable = stateTables.get(restoredMetaInfo.getName()); //important: only create a new table we did not already create it previously if (null == stateTable) { - RegisteredBackendStateMetaInfo registeredBackendStateMetaInfo = - new RegisteredBackendStateMetaInfo<>(metaInfoSerializationProxy); + RegisteredKeyedBackendStateMetaInfo registeredKeyedBackendStateMetaInfo = + new RegisteredKeyedBackendStateMetaInfo<>( + restoredMetaInfo.getStateType(), + restoredMetaInfo.getName(), + restoredMetaInfo.getNamespaceSerializer(), + restoredMetaInfo.getStateSerializer()); - stateTable = newStateTable(registeredBackendStateMetaInfo); - stateTables.put(metaInfoSerializationProxy.getStateName(), stateTable); - kvStatesById.put(numRegisteredKvStates, metaInfoSerializationProxy.getStateName()); + stateTable = newStateTable(registeredKeyedBackendStateMetaInfo); + stateTables.put(restoredMetaInfo.getName(), stateTable); + kvStatesById.put(numRegisteredKvStates, restoredMetaInfo.getName()); ++numRegisteredKvStates; + } else { + // TODO with eager state registration in place, check here for serializer migration strategies } } @@ -410,7 +424,7 @@ private void restorePartitionedState(Collection state) throws Preconditions.checkState(writtenKeyGroupIndex == keyGroupIndex, "Unexpected key-group in restore."); - for (int i = 0; i < metaInfoList.size(); i++) { + for (int i = 0; i < restoredMetaInfos.size(); i++) { int kvStateId = inView.readShort(); StateTable stateTable = stateTables.get(kvStatesById.get(kvStateId)); @@ -509,7 +523,7 @@ public int numStateEntries(Object namespace) { return sum; } - public StateTable newStateTable(RegisteredBackendStateMetaInfo newMetaInfo) { + public StateTable newStateTable(RegisteredKeyedBackendStateMetaInfo newMetaInfo) { return asynchronousSnapshots ? new CopyOnWriteStateTable<>(this, newMetaInfo) : new NestedMapsStateTable<>(this, newMetaInfo); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/NestedMapsStateTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/NestedMapsStateTable.java index 22f344d7c618c..75c31db0f7b39 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/NestedMapsStateTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/NestedMapsStateTable.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; -import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo; +import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo; import org.apache.flink.runtime.state.StateTransformationFunction; import org.apache.flink.util.Preconditions; @@ -63,7 +63,7 @@ public class NestedMapsStateTable extends StateTable { * @param keyContext the key context. * @param metaInfo the meta information for this state table. */ - public NestedMapsStateTable(InternalKeyContext keyContext, RegisteredBackendStateMetaInfo metaInfo) { + public NestedMapsStateTable(InternalKeyContext keyContext, RegisteredKeyedBackendStateMetaInfo metaInfo) { super(keyContext, metaInfo); this.keyGroupOffset = keyContext.getKeyGroupRange().getStartKeyGroup(); diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java index 62fc869736439..c1cdcc39fb659 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTable.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo; +import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo; import org.apache.flink.runtime.state.StateTransformationFunction; import org.apache.flink.util.Preconditions; @@ -42,14 +42,14 @@ public abstract class StateTable { /** * Combined meta information such as name and serializers for this state */ - protected RegisteredBackendStateMetaInfo metaInfo; + protected RegisteredKeyedBackendStateMetaInfo metaInfo; /** * * @param keyContext the key context provides the key scope for all put/get/delete operations. * @param metaInfo the meta information, including the type serializer for state copy-on-write. */ - public StateTable(InternalKeyContext keyContext, RegisteredBackendStateMetaInfo metaInfo) { + public StateTable(InternalKeyContext keyContext, RegisteredKeyedBackendStateMetaInfo metaInfo) { this.keyContext = Preconditions.checkNotNull(keyContext); this.metaInfo = Preconditions.checkNotNull(metaInfo); } @@ -168,11 +168,11 @@ public TypeSerializer getNamespaceSerializer() { return metaInfo.getNamespaceSerializer(); } - public RegisteredBackendStateMetaInfo getMetaInfo() { + public RegisteredKeyedBackendStateMetaInfo getMetaInfo() { return metaInfo; } - public void setMetaInfo(RegisteredBackendStateMetaInfo metaInfo) { + public void setMetaInfo(RegisteredKeyedBackendStateMetaInfo metaInfo) { this.metaInfo = metaInfo; } @@ -186,4 +186,4 @@ public void setMetaInfo(RegisteredBackendStateMetaInfo metaInfo) { @VisibleForTesting public abstract int sizeOfNamespace(Object namespace); -} \ No newline at end of file +} diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTableByKeyGroupReaders.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTableByKeyGroupReaders.java index 53ec3498129d2..d7bc94e27c607 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTableByKeyGroupReaders.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/heap/StateTableByKeyGroupReaders.java @@ -47,7 +47,8 @@ static StateTableByKeyGroupReader readerForVersion(StateTable case 1: return new StateTableByKeyGroupReaderV1<>(table); case 2: - return new StateTableByKeyGroupReaderV2<>(table); + case 3: + return new StateTableByKeyGroupReaderV2V3<>(table); default: throw new IllegalArgumentException("Unknown version: " + version); } @@ -110,10 +111,10 @@ public void readMappingsInKeyGroup(DataInputView inView, int keyGroupId) throws } } - private static final class StateTableByKeyGroupReaderV2 + private static final class StateTableByKeyGroupReaderV2V3 extends AbstractStateTableByKeyGroupReader { - StateTableByKeyGroupReaderV2(StateTable stateTable) { + StateTableByKeyGroupReaderV2V3(StateTable stateTable) { super(stateTable); } @@ -133,4 +134,4 @@ public void readMappingsInKeyGroup(DataInputView inView, int keyGroupId) throws } } } -} \ No newline at end of file +} diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/query/QueryableStateClientTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/query/QueryableStateClientTest.java index b4d6eb7230bad..8c4e049eac473 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/query/QueryableStateClientTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/query/QueryableStateClientTest.java @@ -34,7 +34,7 @@ import org.apache.flink.runtime.query.netty.message.KvStateRequestSerializer; import org.apache.flink.runtime.state.AbstractKeyedStateBackend; import org.apache.flink.runtime.state.KeyGroupRange; -import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo; +import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo; import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.runtime.state.VoidNamespaceSerializer; import org.apache.flink.runtime.state.heap.HeapValueState; @@ -270,7 +270,7 @@ public void testIntegrationWithKvStateServer() throws Exception { ValueStateDescriptor descriptor = new ValueStateDescriptor<>("any", IntSerializer.INSTANCE); - RegisteredBackendStateMetaInfo registeredBackendStateMetaInfo = new RegisteredBackendStateMetaInfo<>( + RegisteredKeyedBackendStateMetaInfo registeredKeyedBackendStateMetaInfo = new RegisteredKeyedBackendStateMetaInfo<>( descriptor.getType(), descriptor.getName(), VoidNamespaceSerializer.INSTANCE, @@ -279,7 +279,7 @@ public void testIntegrationWithKvStateServer() throws Exception { // Register state HeapValueState kvState = new HeapValueState<>( descriptor, - new NestedMapsStateTable(keyedStateBackend, registeredBackendStateMetaInfo), + new NestedMapsStateTable(keyedStateBackend, registeredKeyedBackendStateMetaInfo), IntSerializer.INSTANCE, VoidNamespaceSerializer.INSTANCE); 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 c04ed8cf39562..50ca159420dce 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 @@ -97,7 +97,7 @@ public void testRegisterStatesWithoutTypeSerializer() throws Exception { assertEquals(2, operatorStateBackend.getRegisteredStateNames().size()); // make sure that type registrations are forwarded - TypeSerializer serializer = ((PartitionableListState) listState).getPartitionStateSerializer(); + TypeSerializer serializer = ((PartitionableListState) listState).getStateMetaInfo().getPartitionStateSerializer(); assertTrue(serializer instanceof KryoSerializer); assertTrue(((KryoSerializer) serializer).getKryo().getSerializer(registeredType) instanceof com.esotericsoftware.kryo.serializers.JavaSerializer); 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 0dbe2eb15855d..02b4d6216d909 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,6 +20,7 @@ import org.apache.flink.api.common.state.StateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSerializationProxy; 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; @@ -29,10 +30,21 @@ import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.junit.Assert; import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import java.io.IOException; import java.util.ArrayList; import java.util.List; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({KeyedBackendStateMetaInfoSnapshotReaderWriters.class, OperatorBackendStateMetaInfoSnapshotReaderWriters.class}) public class SerializationProxiesTest { @Test @@ -42,14 +54,14 @@ public void testKeyedBackendSerializationProxyRoundtrip() throws Exception { TypeSerializer namespaceSerializer = LongSerializer.INSTANCE; TypeSerializer stateSerializer = DoubleSerializer.INSTANCE; - List> stateMetaInfoList = new ArrayList<>(); + List> stateMetaInfoList = new ArrayList<>(); - stateMetaInfoList.add( - new KeyedBackendSerializationProxy.StateMetaInfo<>(StateDescriptor.Type.VALUE, "a", namespaceSerializer, stateSerializer)); - stateMetaInfoList.add( - new KeyedBackendSerializationProxy.StateMetaInfo<>(StateDescriptor.Type.VALUE, "b", namespaceSerializer, stateSerializer)); - stateMetaInfoList.add( - new KeyedBackendSerializationProxy.StateMetaInfo<>(StateDescriptor.Type.VALUE, "c", namespaceSerializer, stateSerializer)); + stateMetaInfoList.add(new RegisteredKeyedBackendStateMetaInfo<>( + StateDescriptor.Type.VALUE, "a", namespaceSerializer, stateSerializer).snapshot()); + stateMetaInfoList.add(new RegisteredKeyedBackendStateMetaInfo<>( + StateDescriptor.Type.VALUE, "b", namespaceSerializer, stateSerializer).snapshot()); + stateMetaInfoList.add(new RegisteredKeyedBackendStateMetaInfo<>( + StateDescriptor.Type.VALUE, "c", namespaceSerializer, stateSerializer).snapshot()); KeyedBackendSerializationProxy serializationProxy = new KeyedBackendSerializationProxy(keySerializer, stateMetaInfoList); @@ -67,8 +79,8 @@ public void testKeyedBackendSerializationProxyRoundtrip() throws Exception { serializationProxy.read(new DataInputViewStreamWrapper(in)); } - Assert.assertEquals(keySerializer, serializationProxy.getKeySerializerProxy().getTypeSerializer()); - Assert.assertEquals(stateMetaInfoList, serializationProxy.getNamedStateSerializationProxies()); + Assert.assertEquals(keySerializer, serializationProxy.getKeySerializer()); + Assert.assertEquals(stateMetaInfoList, serializationProxy.getStateMetaInfoSnapshots()); } @Test @@ -78,41 +90,79 @@ public void testKeyedStateMetaInfoSerialization() throws Exception { TypeSerializer namespaceSerializer = LongSerializer.INSTANCE; TypeSerializer stateSerializer = DoubleSerializer.INSTANCE; - KeyedBackendSerializationProxy.StateMetaInfo metaInfo = - new KeyedBackendSerializationProxy.StateMetaInfo<>(StateDescriptor.Type.VALUE, name, namespaceSerializer, stateSerializer); + RegisteredKeyedBackendStateMetaInfo.Snapshot metaInfo = new RegisteredKeyedBackendStateMetaInfo<>( + StateDescriptor.Type.VALUE, name, namespaceSerializer, stateSerializer).snapshot(); byte[] serialized; try (ByteArrayOutputStreamWithPos out = new ByteArrayOutputStreamWithPos()) { - metaInfo.write(new DataOutputViewStreamWrapper(out)); + KeyedBackendStateMetaInfoSnapshotReaderWriters + .getWriterForVersion(KeyedBackendSerializationProxy.VERSION, metaInfo) + .writeStateMetaInfo(new DataOutputViewStreamWrapper(out)); + serialized = out.toByteArray(); } - metaInfo = new KeyedBackendSerializationProxy.StateMetaInfo<>(Thread.currentThread().getContextClassLoader()); - try (ByteArrayInputStreamWithPos in = new ByteArrayInputStreamWithPos(serialized)) { - metaInfo.read(new DataInputViewStreamWrapper(in)); + metaInfo = KeyedBackendStateMetaInfoSnapshotReaderWriters + .getReaderForVersion(KeyedBackendSerializationProxy.VERSION, Thread.currentThread().getContextClassLoader()) + .readStateMetaInfo(new DataInputViewStreamWrapper(in)); } - Assert.assertEquals(name, metaInfo.getStateName()); + Assert.assertEquals(name, metaInfo.getName()); } + @Test + public void testKeyedStateMetaInfoReadSerializerFailureResilience() throws Exception { + String name = "test"; + TypeSerializer namespaceSerializer = LongSerializer.INSTANCE; + TypeSerializer stateSerializer = DoubleSerializer.INSTANCE; + + RegisteredKeyedBackendStateMetaInfo.Snapshot metaInfo = new RegisteredKeyedBackendStateMetaInfo<>( + StateDescriptor.Type.VALUE, name, namespaceSerializer, stateSerializer).snapshot(); + + byte[] serialized; + try (ByteArrayOutputStreamWithPos out = new ByteArrayOutputStreamWithPos()) { + KeyedBackendStateMetaInfoSnapshotReaderWriters + .getWriterForVersion(KeyedBackendSerializationProxy.VERSION, metaInfo) + .writeStateMetaInfo(new DataOutputViewStreamWrapper(out)); + + serialized = out.toByteArray(); + } + + // mock failure when deserializing serializer + TypeSerializerSerializationProxy mockProxy = mock(TypeSerializerSerializationProxy.class); + doThrow(new IOException()).when(mockProxy).read(any(DataInputViewStreamWrapper.class)); + PowerMockito.whenNew(TypeSerializerSerializationProxy.class).withAnyArguments().thenReturn(mockProxy); + + try (ByteArrayInputStreamWithPos in = new ByteArrayInputStreamWithPos(serialized)) { + metaInfo = KeyedBackendStateMetaInfoSnapshotReaderWriters + .getReaderForVersion(KeyedBackendSerializationProxy.VERSION, Thread.currentThread().getContextClassLoader()) + .readStateMetaInfo(new DataInputViewStreamWrapper(in)); + } + + Assert.assertEquals(name, metaInfo.getName()); + Assert.assertEquals(null, metaInfo.getNamespaceSerializer()); + Assert.assertEquals(null, metaInfo.getStateSerializer()); + Assert.assertEquals(namespaceSerializer.snapshotConfiguration(), metaInfo.getNamespaceSerializerConfigSnapshot()); + Assert.assertEquals(stateSerializer.snapshotConfiguration(), metaInfo.getStateSerializerConfigSnapshot()); + } @Test public void testOperatorBackendSerializationProxyRoundtrip() throws Exception { TypeSerializer stateSerializer = DoubleSerializer.INSTANCE; - List> stateMetaInfoList = new ArrayList<>(); + List> stateMetaInfoSnapshots = new ArrayList<>(); - stateMetaInfoList.add( - new OperatorBackendSerializationProxy.StateMetaInfo<>("a", stateSerializer, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); - stateMetaInfoList.add( - new OperatorBackendSerializationProxy.StateMetaInfo<>("b", stateSerializer, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE)); - stateMetaInfoList.add( - new OperatorBackendSerializationProxy.StateMetaInfo<>("c", stateSerializer, OperatorStateHandle.Mode.BROADCAST)); + stateMetaInfoSnapshots.add(new RegisteredOperatorBackendStateMetaInfo<>( + "a", stateSerializer, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE).snapshot()); + stateMetaInfoSnapshots.add(new RegisteredOperatorBackendStateMetaInfo<>( + "b", stateSerializer, OperatorStateHandle.Mode.SPLIT_DISTRIBUTE).snapshot()); + stateMetaInfoSnapshots.add(new RegisteredOperatorBackendStateMetaInfo<>( + "c", stateSerializer, OperatorStateHandle.Mode.BROADCAST).snapshot()); OperatorBackendSerializationProxy serializationProxy = - new OperatorBackendSerializationProxy(stateMetaInfoList); + new OperatorBackendSerializationProxy(stateMetaInfoSnapshots); byte[] serialized; try (ByteArrayOutputStreamWithPos out = new ByteArrayOutputStreamWithPos()) { @@ -127,7 +177,7 @@ public void testOperatorBackendSerializationProxyRoundtrip() throws Exception { serializationProxy.read(new DataInputViewStreamWrapper(in)); } - Assert.assertEquals(stateMetaInfoList, serializationProxy.getNamedStateSerializationProxies()); + Assert.assertEquals(stateMetaInfoSnapshots, serializationProxy.getStateMetaInfoSnapshots()); } @Test @@ -136,22 +186,60 @@ public void testOperatorStateMetaInfoSerialization() throws Exception { String name = "test"; TypeSerializer stateSerializer = DoubleSerializer.INSTANCE; - OperatorBackendSerializationProxy.StateMetaInfo metaInfo = - new OperatorBackendSerializationProxy.StateMetaInfo<>(name, stateSerializer, OperatorStateHandle.Mode.BROADCAST); + RegisteredOperatorBackendStateMetaInfo.Snapshot metaInfo = + new RegisteredOperatorBackendStateMetaInfo<>( + name, stateSerializer, OperatorStateHandle.Mode.BROADCAST).snapshot(); + + byte[] serialized; + try (ByteArrayOutputStreamWithPos out = new ByteArrayOutputStreamWithPos()) { + OperatorBackendStateMetaInfoSnapshotReaderWriters + .getWriterForVersion(OperatorBackendSerializationProxy.VERSION, metaInfo) + .writeStateMetaInfo(new DataOutputViewStreamWrapper(out)); + + serialized = out.toByteArray(); + } + + try (ByteArrayInputStreamWithPos in = new ByteArrayInputStreamWithPos(serialized)) { + metaInfo = OperatorBackendStateMetaInfoSnapshotReaderWriters + .getReaderForVersion(OperatorBackendSerializationProxy.VERSION, Thread.currentThread().getContextClassLoader()) + .readStateMetaInfo(new DataInputViewStreamWrapper(in)); + } + + Assert.assertEquals(name, metaInfo.getName()); + } + + @Test + public void testOperatorStateMetaInfoReadSerializerFailureResilience() throws Exception { + String name = "test"; + TypeSerializer stateSerializer = DoubleSerializer.INSTANCE; + + RegisteredOperatorBackendStateMetaInfo.Snapshot metaInfo = + new RegisteredOperatorBackendStateMetaInfo<>( + name, stateSerializer, OperatorStateHandle.Mode.BROADCAST).snapshot(); byte[] serialized; try (ByteArrayOutputStreamWithPos out = new ByteArrayOutputStreamWithPos()) { - metaInfo.write(new DataOutputViewStreamWrapper(out)); + OperatorBackendStateMetaInfoSnapshotReaderWriters + .getWriterForVersion(OperatorBackendSerializationProxy.VERSION, metaInfo) + .writeStateMetaInfo(new DataOutputViewStreamWrapper(out)); + serialized = out.toByteArray(); } - metaInfo = new OperatorBackendSerializationProxy.StateMetaInfo<>(Thread.currentThread().getContextClassLoader()); + // mock failure when deserializing serializer + TypeSerializerSerializationProxy mockProxy = mock(TypeSerializerSerializationProxy.class); + doThrow(new IOException()).when(mockProxy).read(any(DataInputViewStreamWrapper.class)); + PowerMockito.whenNew(TypeSerializerSerializationProxy.class).withAnyArguments().thenReturn(mockProxy); try (ByteArrayInputStreamWithPos in = new ByteArrayInputStreamWithPos(serialized)) { - metaInfo.read(new DataInputViewStreamWrapper(in)); + metaInfo = OperatorBackendStateMetaInfoSnapshotReaderWriters + .getReaderForVersion(OperatorBackendSerializationProxy.VERSION, Thread.currentThread().getContextClassLoader()) + .readStateMetaInfo(new DataInputViewStreamWrapper(in)); } Assert.assertEquals(name, metaInfo.getName()); + Assert.assertEquals(null, metaInfo.getPartitionStateSerializer()); + Assert.assertEquals(stateSerializer.snapshotConfiguration(), metaInfo.getPartitionStateSerializerConfigSnapshot()); } /** @@ -171,4 +259,4 @@ public void testFixTypeOrder() { Assert.assertEquals(5, StateDescriptor.Type.AGGREGATING.ordinal()); Assert.assertEquals(6, StateDescriptor.Type.MAP.ordinal()); } -} \ No newline at end of file +} 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 6a281099c77b3..96025fe16552b 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 @@ -43,6 +43,8 @@ import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.typeutils.GenericTypeInfo; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.api.java.typeutils.runtime.PojoSerializer; import org.apache.flink.api.java.typeutils.runtime.kryo.JavaSerializer; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; @@ -67,12 +69,10 @@ import org.apache.flink.util.IOUtils; import org.apache.flink.util.TestLogger; import org.junit.Assert; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; -import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; import java.util.Collections; @@ -567,7 +567,6 @@ public void testKryoRegisteringRestoreResilienceWithDefaultSerializer() throws E * * @throws Exception expects {@link ExpectedKryoTestException} to be thrown. */ - @Ignore @Test public void testKryoRegisteringRestoreResilienceWithRegisteredSerializer() throws Exception { CheckpointStreamFactory streamFactory = createStreamFactory(); @@ -644,6 +643,139 @@ public void testKryoRegisteringRestoreResilienceWithRegisteredSerializer() throw state.value(); } + @Test + public void testKryoRestoreResilienceWithDifferentRegistrationOrder() throws Exception { + CheckpointStreamFactory streamFactory = createStreamFactory(); + Environment env = new DummyEnvironment("test", 1, 0); + + // register A first then B + env.getExecutionConfig().registerKryoType(TestNestedPojoClassA.class); + env.getExecutionConfig().registerKryoType(TestNestedPojoClassB.class); + + AbstractKeyedStateBackend backend = createKeyedBackend(IntSerializer.INSTANCE, env); + + TypeInformation pojoType = new GenericTypeInfo<>(TestPojo.class); + + // make sure that we are in fact using the KryoSerializer + assertTrue(pojoType.createSerializer(env.getExecutionConfig()) instanceof KryoSerializer); + + ValueStateDescriptor kvId = new ValueStateDescriptor<>("id", pojoType); + ValueState state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId); + + // ============== create snapshot of current configuration ============== + + // make some more modifications + backend.setCurrentKey(1); + state.update(new TestPojo("u1", 1, new TestNestedPojoClassA(1.0, 2), new TestNestedPojoClassB(2.3, "foo"))); + + backend.setCurrentKey(2); + state.update(new TestPojo("u2", 2, new TestNestedPojoClassA(2.0, 5), new TestNestedPojoClassB(3.1, "bar"))); + + KeyedStateHandle snapshot = runSnapshot(backend.snapshot( + 682375462378L, + 2, + streamFactory, + CheckpointOptions.forFullCheckpoint())); + + backend.dispose(); + + // ========== restore snapshot, with a different registration order in the configuration ========== + + env = new DummyEnvironment("test", 1, 0); + + env.getExecutionConfig().registerKryoType(TestNestedPojoClassB.class); // this time register B first + env.getExecutionConfig().registerKryoType(TestNestedPojoClassA.class); + + backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot, env); + + snapshot.discardState(); + + // re-initialize to ensure that we create the KryoSerializer from scratch, otherwise + // initializeSerializerUnlessSet would not pick up our new config + kvId = new ValueStateDescriptor<>("id", pojoType); + state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId); + + backend.setCurrentKey(1); + + // update to test state backends that eagerly serialize, such as RocksDB + state.update(new TestPojo("u1", 11, new TestNestedPojoClassA(22.1, 12), new TestNestedPojoClassB(1.23, "foobar"))); + + // this tests backends that lazily serialize, such as memory state backend + runSnapshot(backend.snapshot( + 682375462378L, + 2, + streamFactory, + CheckpointOptions.forFullCheckpoint())); + + backend.dispose(); + } + + @Test + public void testPojoRestoreResilienceWithDifferentRegistrationOrder() throws Exception { + CheckpointStreamFactory streamFactory = createStreamFactory(); + Environment env = new DummyEnvironment("test", 1, 0); + + // register A first then B + env.getExecutionConfig().registerPojoType(TestNestedPojoClassA.class); + env.getExecutionConfig().registerPojoType(TestNestedPojoClassB.class); + + AbstractKeyedStateBackend backend = createKeyedBackend(IntSerializer.INSTANCE, env); + + TypeInformation pojoType = TypeExtractor.getForClass(TestPojo.class); + + // make sure that we are in fact using the PojoSerializer + assertTrue(pojoType.createSerializer(env.getExecutionConfig()) instanceof PojoSerializer); + + ValueStateDescriptor kvId = new ValueStateDescriptor<>("id", pojoType); + ValueState state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId); + + // ============== create snapshot of current configuration ============== + + // make some more modifications + backend.setCurrentKey(1); + state.update(new TestPojo("u1", 1, new TestNestedPojoClassA(1.0, 2), new TestNestedPojoClassB(2.3, "foo"))); + + backend.setCurrentKey(2); + state.update(new TestPojo("u2", 2, new TestNestedPojoClassA(2.0, 5), new TestNestedPojoClassB(3.1, "bar"))); + + KeyedStateHandle snapshot = runSnapshot(backend.snapshot( + 682375462378L, + 2, + streamFactory, + CheckpointOptions.forFullCheckpoint())); + + backend.dispose(); + + // ========== restore snapshot, with a different registration order in the configuration ========== + + env = new DummyEnvironment("test", 1, 0); + + env.getExecutionConfig().registerPojoType(TestNestedPojoClassB.class); // this time register B first + env.getExecutionConfig().registerPojoType(TestNestedPojoClassA.class); + + backend = restoreKeyedBackend(IntSerializer.INSTANCE, snapshot, env); + + snapshot.discardState(); + + // re-initialize to ensure that we create the PojoSerializer from scratch, otherwise + // initializeSerializerUnlessSet would not pick up our new config + kvId = new ValueStateDescriptor<>("id", pojoType); + state = backend.getPartitionedState(VoidNamespace.INSTANCE, VoidNamespaceSerializer.INSTANCE, kvId); + + backend.setCurrentKey(1); + + // update to test state backends that eagerly serialize, such as RocksDB + state.update(new TestPojo("u1", 11, new TestNestedPojoClassA(22.1, 12), new TestNestedPojoClassB(1.23, "foobar"))); + + // this tests backends that lazily serialize, such as memory state backend + runSnapshot(backend.snapshot( + 682375462378L, + 2, + streamFactory, + CheckpointOptions.forFullCheckpoint())); + + backend.dispose(); + } @Test @SuppressWarnings("unchecked") @@ -1698,8 +1830,8 @@ public void testValueStateRestoreWithWrongSerializers() { state.value(); fail("should recognize wrong serializers"); - } catch (IOException e) { - if (!e.getMessage().contains("Trying to access state using wrong")) { + } catch (RuntimeException e) { + if (!e.getMessage().contains("State migration currently isn't supported")) { fail("wrong exception " + e); } // expected @@ -1749,8 +1881,8 @@ public void testListStateRestoreWithWrongSerializers() { state.get(); fail("should recognize wrong serializers"); - } catch (IOException e) { - if (!e.getMessage().contains("Trying to access state using wrong")) { + } catch (RuntimeException e) { + if (!e.getMessage().contains("State migration currently isn't supported")) { fail("wrong exception " + e); } // expected @@ -1802,8 +1934,8 @@ public void testReducingStateRestoreWithWrongSerializers() { state.get(); fail("should recognize wrong serializers"); - } catch (IOException e) { - if (!e.getMessage().contains("Trying to access state using wrong ")) { + } catch (RuntimeException e) { + if (!e.getMessage().contains("State migration currently isn't supported")) { fail("wrong exception " + e); } // expected @@ -1853,8 +1985,8 @@ public void testMapStateRestoreWithWrongSerializers() { state.entries(); fail("should recognize wrong serializers"); - } catch (IOException e) { - if (!e.getMessage().contains("Trying to access state using wrong ")) { + } catch (RuntimeException e) { + if (!e.getMessage().contains("State migration currently isn't supported")) { fail("wrong exception " + e); } // expected @@ -2384,15 +2516,27 @@ private KeyedStateHandle runSnapshot(RunnableFuture snapshotRu return snapshotRunnableFuture.get(); } - private static class TestPojo implements Serializable { + public static class TestPojo implements Serializable { private String strField; private Integer intField; + private TestNestedPojoClassA kryoClassAField; + private TestNestedPojoClassB kryoClassBField; + public TestPojo() {} public TestPojo(String strField, Integer intField) { this.strField = strField; this.intField = intField; + this.kryoClassAField = null; + this.kryoClassBField = null; + } + + public TestPojo(String strField, Integer intField, TestNestedPojoClassA classAField, TestNestedPojoClassB classBfield) { + this.strField = strField; + this.intField = intField; + this.kryoClassAField = classAField; + this.kryoClassBField = classBfield; } public String getStrField() { @@ -2411,6 +2555,22 @@ public void setIntField(Integer intField) { this.intField = intField; } + public TestNestedPojoClassA getKryoClassAField() { + return kryoClassAField; + } + + public void setKryoClassAField(TestNestedPojoClassA kryoClassAField) { + this.kryoClassAField = kryoClassAField; + } + + public TestNestedPojoClassB getKryoClassBField() { + return kryoClassBField; + } + + public void setKryoClassBField(TestNestedPojoClassB kryoClassBField) { + this.kryoClassBField = kryoClassBField; + } + @Override public String toString() { return "TestPojo{" + @@ -2426,14 +2586,133 @@ public boolean equals(Object o) { TestPojo testPojo = (TestPojo) o; - if (!strField.equals(testPojo.strField)) return false; - return intField.equals(testPojo.intField); + return strField.equals(testPojo.strField) + && intField.equals(testPojo.intField) + && ((kryoClassAField == null && testPojo.kryoClassAField == null) || kryoClassAField.equals(testPojo.kryoClassAField)) + && ((kryoClassBField == null && testPojo.kryoClassBField == null) || kryoClassBField.equals(testPojo.kryoClassBField)); } @Override public int hashCode() { int result = strField.hashCode(); result = 31 * result + intField.hashCode(); + + if (kryoClassAField != null) { + result = 31 * result + kryoClassAField.hashCode(); + } + + if (kryoClassBField != null) { + result = 31 * result + kryoClassBField.hashCode(); + } + + return result; + } + } + + public static class TestNestedPojoClassA implements Serializable { + private Double doubleField; + private Integer intField; + + public TestNestedPojoClassA() {} + + public TestNestedPojoClassA(Double doubleField, Integer intField) { + this.doubleField = doubleField; + this.intField = intField; + } + + public Double getDoubleField() { + return doubleField; + } + + public void setDoubleField(Double doubleField) { + this.doubleField = doubleField; + } + + public Integer getIntField() { + return intField; + } + + public void setIntField(Integer intField) { + this.intField = intField; + } + + @Override + public String toString() { + return "TestNestedPojoClassA{" + + "doubleField='" + doubleField + '\'' + + ", intField=" + intField + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + TestNestedPojoClassA testNestedPojoClassA = (TestNestedPojoClassA) o; + + if (!doubleField.equals(testNestedPojoClassA.doubleField)) return false; + return intField.equals(testNestedPojoClassA.intField); + } + + @Override + public int hashCode() { + int result = doubleField.hashCode(); + result = 31 * result + intField.hashCode(); + return result; + } + } + + public static class TestNestedPojoClassB implements Serializable { + private Double doubleField; + private String strField; + + public TestNestedPojoClassB() {} + + public TestNestedPojoClassB(Double doubleField, String strField) { + this.doubleField = doubleField; + this.strField = strField; + } + + public Double getDoubleField() { + return doubleField; + } + + public void setDoubleField(Double doubleField) { + this.doubleField = doubleField; + } + + public String getStrField() { + return strField; + } + + public void setStrField(String strField) { + this.strField = strField; + } + + @Override + public String toString() { + return "TestNestedPojoClassB{" + + "doubleField='" + doubleField + '\'' + + ", strField=" + strField + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + TestNestedPojoClassB testNestedPojoClassB = (TestNestedPojoClassB) o; + + if (!doubleField.equals(testNestedPojoClassB.doubleField)) return false; + return strField.equals(testNestedPojoClassB.strField); + } + + @Override + public int hashCode() { + int result = doubleField.hashCode(); + result = 31 * result + strField.hashCode(); return result; } } 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 69b371fd6f44e..987413750024a 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 @@ -31,7 +31,7 @@ import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.runtime.state.ArrayListSerializer; import org.apache.flink.runtime.state.KeyGroupRange; -import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo; +import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo; import org.apache.flink.runtime.state.StateTransformationFunction; import org.apache.flink.util.TestLogger; import org.junit.Assert; @@ -52,8 +52,8 @@ public class CopyOnWriteStateTableTest extends TestLogger { */ @Test public void testPutGetRemoveContainsTransform() throws Exception { - RegisteredBackendStateMetaInfo> metaInfo = - new RegisteredBackendStateMetaInfo<>( + RegisteredKeyedBackendStateMetaInfo> metaInfo = + new RegisteredKeyedBackendStateMetaInfo<>( StateDescriptor.Type.UNKNOWN, "test", IntSerializer.INSTANCE, @@ -124,8 +124,8 @@ public ArrayList apply(ArrayList previousState, Integer value) */ @Test public void testIncrementalRehash() { - RegisteredBackendStateMetaInfo> metaInfo = - new RegisteredBackendStateMetaInfo<>( + RegisteredKeyedBackendStateMetaInfo> metaInfo = + new RegisteredKeyedBackendStateMetaInfo<>( StateDescriptor.Type.UNKNOWN, "test", IntSerializer.INSTANCE, @@ -169,8 +169,8 @@ public void testIncrementalRehash() { @Test public void testRandomModificationsAndCopyOnWriteIsolation() throws Exception { - final RegisteredBackendStateMetaInfo> metaInfo = - new RegisteredBackendStateMetaInfo<>( + final RegisteredKeyedBackendStateMetaInfo> metaInfo = + new RegisteredKeyedBackendStateMetaInfo<>( StateDescriptor.Type.UNKNOWN, "test", IntSerializer.INSTANCE, @@ -324,8 +324,8 @@ public ArrayList apply(ArrayList previousState, Integer value) */ @Test public void testCopyOnWriteContracts() { - RegisteredBackendStateMetaInfo> metaInfo = - new RegisteredBackendStateMetaInfo<>( + RegisteredKeyedBackendStateMetaInfo> metaInfo = + new RegisteredKeyedBackendStateMetaInfo<>( StateDescriptor.Type.UNKNOWN, "test", IntSerializer.INSTANCE, @@ -399,8 +399,8 @@ public void testSerializerDuplicationInSnapshot() throws IOException { final TestDuplicateSerializer stateSerializer = new TestDuplicateSerializer();; final TestDuplicateSerializer keySerializer = new TestDuplicateSerializer();; - RegisteredBackendStateMetaInfo metaInfo = - new RegisteredBackendStateMetaInfo<>( + RegisteredKeyedBackendStateMetaInfo metaInfo = + new RegisteredKeyedBackendStateMetaInfo<>( StateDescriptor.Type.VALUE, "test", namespaceSerializer, diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/StateTableSnapshotCompatibilityTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/StateTableSnapshotCompatibilityTest.java index 6fd94f774917e..85bc1774b58f9 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/StateTableSnapshotCompatibilityTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/StateTableSnapshotCompatibilityTest.java @@ -27,7 +27,7 @@ import org.apache.flink.runtime.state.ArrayListSerializer; import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyedBackendSerializationProxy; -import org.apache.flink.runtime.state.RegisteredBackendStateMetaInfo; +import org.apache.flink.runtime.state.RegisteredKeyedBackendStateMetaInfo; import org.junit.Assert; import org.junit.Test; @@ -44,8 +44,8 @@ public class StateTableSnapshotCompatibilityTest { @Test public void checkCompatibleSerializationFormats() throws IOException { final Random r = new Random(42); - RegisteredBackendStateMetaInfo> metaInfo = - new RegisteredBackendStateMetaInfo<>( + RegisteredKeyedBackendStateMetaInfo> metaInfo = + new RegisteredKeyedBackendStateMetaInfo<>( StateDescriptor.Type.UNKNOWN, "test", IntSerializer.INSTANCE, From 81dd0eb6301723fc2ce5b0358670f6e43b84eb8a Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sun, 7 May 2017 19:51:12 +0800 Subject: [PATCH 4/5] [FLINK-6425] Change naming of MigrationStrategy to CompatibilityDecision --- .../typeutils/runtime/WritableSerializer.java | 8 ++-- .../state/RocksDBKeyedStateBackend.java | 18 ++++---- .../streaming/state/RocksDBStateBackend.java | 2 +- ...rategy.java => CompatibilityDecision.java} | 42 +++++++------------ .../api/common/typeutils/TypeSerializer.java | 25 +++++------ .../TypeSerializerSerializationProxy.java | 2 +- .../common/typeutils/base/EnumSerializer.java | 8 ++-- .../base/GenericArraySerializer.java | 18 ++++---- .../common/typeutils/base/ListSerializer.java | 18 ++++---- .../common/typeutils/base/MapSerializer.java | 22 +++++----- .../base/TypeSerializerSingleton.java | 8 ++-- .../typeutils/runtime/AvroSerializer.java | 10 ++--- .../runtime/CopyableValueSerializer.java | 8 ++-- .../typeutils/runtime/EitherSerializer.java | 22 +++++----- .../typeutils/runtime/PojoSerializer.java | 26 ++++++------ .../java/typeutils/runtime/RowSerializer.java | 18 ++++---- .../runtime/TupleSerializerBase.java | 12 +++--- .../typeutils/runtime/ValueSerializer.java | 8 ++-- .../runtime/kryo/KryoSerializer.java | 10 ++--- .../common/typeutils/SerializerTestBase.java | 2 +- .../TypeSerializerConfigSnapshotTest.java | 5 +-- .../typeutils/base/EnumSerializerTest.java | 8 ++-- .../typeutils/runtime/PojoSerializerTest.java | 12 +++--- .../kryo/KryoSerializerMigrationTest.java | 6 +-- .../java/io/CollectionInputFormatTest.java | 4 +- .../cep/NonDuplicatingTypeSerializer.java | 4 +- .../AbstractKeyedCEPPatternOperator.java | 18 ++++---- .../table/runtime/types/CRowSerializer.scala | 18 ++++---- .../MigrationNamespaceSerializerProxy.java | 6 +-- .../runtime/state/ArrayListSerializer.java | 18 ++++---- .../runtime/state/HashMapSerializer.java | 22 +++++----- .../state/KeyedBackendSerializationProxy.java | 2 +- .../OperatorBackendSerializationProxy.java | 2 +- .../testutils/types/IntListSerializer.java | 4 +- .../testutils/types/IntPairSerializer.java | 4 +- .../testutils/types/StringPairSerializer.java | 4 +- .../state/heap/CopyOnWriteStateTableTest.java | 4 +- .../recordutils/RecordSerializer.java | 4 +- .../scala/typeutils/EitherSerializer.scala | 22 +++++----- .../scala/typeutils/EnumValueSerializer.scala | 14 +++---- .../scala/typeutils/NothingSerializer.scala | 6 +-- .../scala/typeutils/OptionSerializer.scala | 16 +++---- .../typeutils/TraversableSerializer.scala | 6 +-- .../api/scala/typeutils/TrySerializer.scala | 10 ++--- .../MultiplexingStreamRecordSerializer.java | 18 ++++---- .../streamrecord/StreamRecordSerializer.java | 18 ++++---- .../api/datastream/CoGroupedStreams.java | 4 +- .../api/operators/InternalTimer.java | 4 +- .../streamrecord/StreamElementSerializer.java | 18 ++++---- 49 files changed, 275 insertions(+), 293 deletions(-) rename flink-core/src/main/java/org/apache/flink/api/common/typeutils/{MigrationStrategy.java => CompatibilityDecision.java} (55%) 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 2576cb063fe42..a3c28be7ea0bf 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 @@ -21,8 +21,8 @@ import com.esotericsoftware.kryo.Kryo; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.GenericTypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; @@ -165,13 +165,13 @@ public WritableSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof WritableSerializerConfigSnapshot && typeClass.equals(((WritableSerializerConfigSnapshot) configSnapshot).getTypeClass())) { - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index 57f941087c993..ef255128052c4 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -26,7 +26,7 @@ import org.apache.flink.api.common.state.ReducingStateDescriptor; import org.apache.flink.api.common.state.StateDescriptor; import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSerializationProxy; import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; @@ -1521,15 +1521,15 @@ protected ColumnFamilyHandle getColumnFamily( // in which case we can only simply assume that migration is not required if (restoredMetaInfo.getNamespaceSerializerConfigSnapshot() != null) { - MigrationStrategy namespaceMigrationStrategy = newMetaInfo.getNamespaceSerializer() + CompatibilityDecision namespaceCompatibilityDecision = newMetaInfo.getNamespaceSerializer() .getMigrationStrategyFor(restoredMetaInfo.getNamespaceSerializerConfigSnapshot()); TypeSerializer finalOldNamespaceSerializer; - if (namespaceMigrationStrategy.requireMigration()) { + if (namespaceCompatibilityDecision.requireMigration()) { requireMigration = true; - if (namespaceMigrationStrategy.getFallbackDeserializer() != null) { - finalOldNamespaceSerializer = namespaceMigrationStrategy.getFallbackDeserializer(); + if (namespaceCompatibilityDecision.getConvertDeserializer() != null) { + finalOldNamespaceSerializer = namespaceCompatibilityDecision.getConvertDeserializer(); } else if (restoredMetaInfo.getNamespaceSerializer() != null && !(restoredMetaInfo.getNamespaceSerializer() instanceof MigrationNamespaceSerializerProxy)) { finalOldNamespaceSerializer = restoredMetaInfo.getNamespaceSerializer(); @@ -1541,15 +1541,15 @@ protected ColumnFamilyHandle getColumnFamily( } if (restoredMetaInfo.getStateSerializerConfigSnapshot() != null) { - MigrationStrategy stateMigrationStrategy = newMetaInfo.getStateSerializer() + CompatibilityDecision stateCompatibilityDecision = newMetaInfo.getStateSerializer() .getMigrationStrategyFor(restoredMetaInfo.getStateSerializerConfigSnapshot()); TypeSerializer finalOldStateSerializer; - if (stateMigrationStrategy.requireMigration()) { + if (stateCompatibilityDecision.requireMigration()) { requireMigration = true; - if (stateMigrationStrategy.getFallbackDeserializer() != null) { - finalOldStateSerializer = stateMigrationStrategy.getFallbackDeserializer(); + if (stateCompatibilityDecision.getConvertDeserializer() != null) { + finalOldStateSerializer = stateCompatibilityDecision.getConvertDeserializer(); } else if (restoredMetaInfo.getStateSerializer() != null && !(restoredMetaInfo.getStateSerializer() instanceof TypeSerializerSerializationProxy.ClassNotFoundDummyTypeSerializer)) { finalOldStateSerializer = restoredMetaInfo.getStateSerializer(); diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/migration/contrib/streaming/state/RocksDBStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/migration/contrib/streaming/state/RocksDBStateBackend.java index e5a78b69a500d..695aa12b4221b 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/migration/contrib/streaming/state/RocksDBStateBackend.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/migration/contrib/streaming/state/RocksDBStateBackend.java @@ -75,7 +75,7 @@ public static class FinalSemiAsyncSnapshot { } private static void throwExceptionOnLoadingThisClass() { - throw new RuntimeException("Attempt to migrate RocksDB state created with semi async snapshot mode failed. " + throw new RuntimeException("Attempt to requiresMigration RocksDB state created with semi async snapshot mode failed. " + "Unfortunately, this is not supported. Please create a new savepoint for the job using fully " + "async mode in Flink 1.1 and run migration again with the new savepoint."); } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/MigrationStrategy.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompatibilityDecision.java similarity index 55% rename from flink-core/src/main/java/org/apache/flink/api/common/typeutils/MigrationStrategy.java rename to flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompatibilityDecision.java index c76027b897c5e..4049a0ed11d3c 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/MigrationStrategy.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/CompatibilityDecision.java @@ -21,45 +21,31 @@ import org.apache.flink.annotation.PublicEvolving; /** - * A {@code MigrationStrategy} contains information about how to perform migration of data written + * A {@code CompatibilityDecision} contains information about how to perform migration of data written * by an older serializer so that new serializers can continue to work on them. * * @param the type of the data being migrated. */ @PublicEvolving -public final class MigrationStrategy { +public final class CompatibilityDecision { /** Whether or not migration is required. */ - private final boolean requiresStateMigration; + private final boolean requiresMigration; /** * The fallback deserializer to use, in the case the preceding serializer cannot be found. * *

This is only relevant if migration is required. */ - private final TypeSerializer fallbackDeserializer; + private final TypeSerializer convertDeserializer; /** * Returns a strategy that simply signals that no migration needs to be performed. * * @return a strategy that does not perform migration */ - public static MigrationStrategy noMigration() { - return new MigrationStrategy<>(false, null); - } - - /** - * Returns a strategy that signals migration to be performed, and in the case that the - * preceding serializer cannot be found, a provided fallback deserializer can be - * used. - * - * @param fallbackDeserializer a fallback deserializer that can be used to read old data for the migration - * in the case that the preceding serializer cannot be found. - * - * @return a strategy that performs migration with a fallback deserializer to read old data. - */ - public static MigrationStrategy migrateWithFallbackDeserializer(TypeSerializer fallbackDeserializer) { - return new MigrationStrategy<>(true, fallbackDeserializer); + public static CompatibilityDecision compatible() { + return new CompatibilityDecision<>(false, null); } /** @@ -68,20 +54,20 @@ public static MigrationStrategy migrateWithFallbackDeserializer(TypeSeria * * @return a strategy that performs migration, without a fallback deserializer. */ - public static MigrationStrategy migrate() { - return new MigrationStrategy<>(true, null); + public static CompatibilityDecision requiresMigration(TypeSerializer fallbackDeserializer) { + return new CompatibilityDecision<>(true, fallbackDeserializer); } - private MigrationStrategy(boolean requiresStateMigration, TypeSerializer fallbackDeserializer) { - this.requiresStateMigration = requiresStateMigration; - this.fallbackDeserializer = fallbackDeserializer; + private CompatibilityDecision(boolean requiresMigration, TypeSerializer convertDeserializer) { + this.requiresMigration = requiresMigration; + this.convertDeserializer = convertDeserializer; } - public TypeSerializer getFallbackDeserializer() { - return fallbackDeserializer; + public TypeSerializer getConvertDeserializer() { + return convertDeserializer; } public boolean requireMigration() { - return requiresStateMigration; + return requiresMigration; } } 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 6effc1e3981b9..a52c975a55a50 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 @@ -199,30 +199,27 @@ public abstract class TypeSerializer implements Serializable { * *

Implementations need to return the resolved migration strategy. The strategy can be one of the following: *

    - *
  • {@link MigrationStrategy#noMigration()}: this signals Flink that this serializer is compatible, or + *
  • {@link CompatibilityDecision#compatible()}: this signals Flink that this serializer is compatible, or * has been reconfigured to be compatible, to continue reading old data, and that the * serialization schema remains the same. No migration needs to be performed.
  • * - *
  • {@link MigrationStrategy#migrateWithFallbackDeserializer(TypeSerializer)}: this signals Flink that + *
  • {@link CompatibilityDecision#requiresMigration(TypeSerializer)}: this signals Flink that * migration needs to be performed, because this serializer is not compatible, or cannot be reconfigured to be * compatible, for old data. Furthermore, in the case that the preceding serializer cannot be found or - * restored to read the old data, the provided fallback deserializer can be used.
  • - * - *
  • {@link MigrationStrategy#migrate()}: this signals Flink that migration needs to be performed, because - * this serializer is not compatible, or cannot be reconfigured to be compatible, for old data.
  • + * restored to read the old data, the provided convert deserializer can be used. *
* *

This method is guaranteed to only be invoked if the preceding serializer's configuration snapshot is not the * singleton {@link ForwardCompatibleSerializationFormatConfig#INSTANCE} configuration. In such cases, Flink always - * assume that the migration strategy is {@link MigrationStrategy#migrate()}. + * assume that the migration strategy is {@link CompatibilityDecision#requiresMigration(TypeSerializer)}. * - * @see MigrationStrategy + * @see CompatibilityDecision * * @param configSnapshot configuration snapshot of a preceding serializer for the same managed state * * @return the result of the reconfiguration. */ - protected abstract MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot); + protected abstract CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot); /** * Get the migration strategy to use this serializer based on the configuration snapshot of a preceding @@ -232,23 +229,23 @@ public abstract class TypeSerializer implements Serializable { *

This method is not part of the public user-facing API, and cannot be overriden. External operations * providing a configuration snapshot of preceding serializer can only do so through this method. * - *

This method always assumes that the migration strategy is {@link MigrationStrategy#noMigration()} if + *

This method always assumes that the migration strategy is {@link CompatibilityDecision#compatible()} if * the provided configuration snapshot is the singleton {@link ForwardCompatibleSerializationFormatConfig#INSTANCE}. * Otherwise, the configuration snapshot is provided to the actual - * {@link #getMigrationStrategy(TypeSerializerConfigSnapshot)} (TypeSerializerConfigSnapshot)} implementation. + * {@link #ensureCompatibility(TypeSerializerConfigSnapshot)} (TypeSerializerConfigSnapshot)} implementation. * * @param configSnapshot configuration snapshot of a preceding serializer for the same managed state * * @return the result of the reconfiguration. */ @Internal - public final MigrationStrategy getMigrationStrategyFor(TypeSerializerConfigSnapshot configSnapshot) { + public final CompatibilityDecision getMigrationStrategyFor(TypeSerializerConfigSnapshot configSnapshot) { // reference equality is viable here, because the forward compatible // marker config will always be explicitly restored with the singleton instance if (configSnapshot != ForwardCompatibleSerializationFormatConfig.INSTANCE) { - return getMigrationStrategy(configSnapshot); + return ensureCompatibility(configSnapshot); } else { - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java index f4ccca70cedfa..04e02cd4c5057 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java @@ -212,7 +212,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision 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/EnumSerializer.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/EnumSerializer.java index 20ff638a147d2..bd2be6d2a52bf 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/EnumSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/EnumSerializer.java @@ -28,8 +28,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.GenericTypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream; @@ -179,7 +179,7 @@ public EnumSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof EnumSerializerConfigSnapshot) { final EnumSerializerConfigSnapshot config = (EnumSerializerConfigSnapshot) configSnapshot; @@ -201,11 +201,11 @@ protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot i++; } - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } /** 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 b8ffd4a158246..8011567076f84 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/GenericArraySerializer.java @@ -22,7 +22,7 @@ import java.lang.reflect.Array; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; @@ -199,29 +199,29 @@ public GenericArraySerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof GenericArraySerializerConfigSnapshot) { final GenericArraySerializerConfigSnapshot config = (GenericArraySerializerConfigSnapshot) configSnapshot; if (componentClass.equals(config.getComponentClass())) { - MigrationStrategy strategy = componentSerializer.getMigrationStrategyFor( + CompatibilityDecision strategy = componentSerializer.getMigrationStrategyFor( config.getSingleNestedSerializerConfigSnapshot()); if (strategy.requireMigration()) { - if (strategy.getFallbackDeserializer() != null) { - return MigrationStrategy.migrateWithFallbackDeserializer( + if (strategy.getConvertDeserializer() != null) { + return CompatibilityDecision.requiresMigration( new GenericArraySerializer<>( componentClass, - strategy.getFallbackDeserializer())); + strategy.getConvertDeserializer())); } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } else { - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } } - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } 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 082e11562b360..04b071aa14fb3 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/ListSerializer.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.typeutils.base; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; @@ -179,23 +179,23 @@ public CollectionSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof CollectionSerializerConfigSnapshot) { - MigrationStrategy strategy = elementSerializer.getMigrationStrategyFor( + CompatibilityDecision strategy = elementSerializer.getMigrationStrategyFor( ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); if (strategy.requireMigration()) { - if (strategy.getFallbackDeserializer() != null) { - return MigrationStrategy.migrateWithFallbackDeserializer( - new ListSerializer<>(strategy.getFallbackDeserializer())); + if (strategy.getConvertDeserializer() != null) { + return CompatibilityDecision.requiresMigration( + new ListSerializer<>(strategy.getConvertDeserializer())); } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } else { - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } } 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 8cb6836dddc85..e3bb6a72dae28 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/base/MapSerializer.java @@ -19,7 +19,7 @@ package org.apache.flink.api.common.typeutils.base; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; @@ -207,28 +207,28 @@ public MapSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof MapSerializerConfigSnapshot) { TypeSerializerConfigSnapshot[] keyValueSerializerConfigSnapshots = ((MapSerializerConfigSnapshot) configSnapshot).getNestedSerializerConfigSnapshots(); - MigrationStrategy keyStrategy = keySerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[0]); - MigrationStrategy valueStrategy = valueSerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[1]); + CompatibilityDecision keyStrategy = keySerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[0]); + CompatibilityDecision valueStrategy = valueSerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[1]); if (keyStrategy.requireMigration() || valueStrategy.requireMigration()) { - if (keyStrategy.getFallbackDeserializer() != null && valueStrategy.getFallbackDeserializer() != null) { - return MigrationStrategy.migrateWithFallbackDeserializer( + if (keyStrategy.getConvertDeserializer() != null && valueStrategy.getConvertDeserializer() != null) { + return CompatibilityDecision.requiresMigration( new MapSerializer<>( - keyStrategy.getFallbackDeserializer(), - valueStrategy.getFallbackDeserializer())); + keyStrategy.getConvertDeserializer(), + valueStrategy.getConvertDeserializer())); } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } else { - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } } 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 9b7371a0dc721..f86c57c0e0a5f 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 @@ -20,7 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.ParameterlessTypeSerializerConfig; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; @@ -59,14 +59,14 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - public MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof ParameterlessTypeSerializerConfig && isCompatibleSerializationFormatIdentifier( ((ParameterlessTypeSerializerConfig) configSnapshot).getSerializationFormatIdentifier())) { - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java index abae49f39fa5e..c1ee2eca21906 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java @@ -30,7 +30,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.java.typeutils.runtime.kryo.Serializers; @@ -225,7 +225,7 @@ public AvroSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof AvroSerializerConfigSnapshot) { final AvroSerializerConfigSnapshot config = (AvroSerializerConfigSnapshot) configSnapshot; @@ -238,18 +238,18 @@ protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot for (Map.Entry reconfiguredRegistrationEntry : kryoRegistrations.entrySet()) { if (reconfiguredRegistrationEntry.getValue().isDummy()) { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } this.kryoRegistrations = oldRegistrations; - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } // ends up here if the preceding serializer is not // the ValueSerializer, or serialized data type has changed - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } public static class AvroSerializerConfigSnapshot extends KryoRegistrationSerializerConfigSnapshot { 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 2a29a3f379207..b4d69b6a215fc 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/CopyableValueSerializer.java @@ -21,8 +21,8 @@ import java.io.IOException; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.GenericTypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; @@ -142,12 +142,12 @@ public CopyableValueSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof CopyableValueSerializerConfigSnapshot && valueClass.equals(((CopyableValueSerializerConfigSnapshot) configSnapshot).getTypeClass())) { - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } 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 aad11f69c2f30..26c11c30e08be 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/EitherSerializer.java @@ -19,7 +19,7 @@ package org.apache.flink.api.java.typeutils.runtime; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; @@ -198,28 +198,28 @@ public EitherSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof EitherSerializerConfigSnapshot) { TypeSerializerConfigSnapshot[] leftRightSerializerConfigSnapshots = ((EitherSerializerConfigSnapshot) configSnapshot).getNestedSerializerConfigSnapshots(); - MigrationStrategy leftStrategy = leftSerializer.getMigrationStrategyFor(leftRightSerializerConfigSnapshots[0]); - MigrationStrategy rightStrategy = rightSerializer.getMigrationStrategyFor(leftRightSerializerConfigSnapshots[1]); + CompatibilityDecision leftStrategy = leftSerializer.getMigrationStrategyFor(leftRightSerializerConfigSnapshots[0]); + CompatibilityDecision rightStrategy = rightSerializer.getMigrationStrategyFor(leftRightSerializerConfigSnapshots[1]); if (leftStrategy.requireMigration() || rightStrategy.requireMigration()) { - if (leftStrategy.getFallbackDeserializer() != null && rightStrategy.getFallbackDeserializer() != null) { - return MigrationStrategy.migrateWithFallbackDeserializer( + if (leftStrategy.getConvertDeserializer() != null && rightStrategy.getConvertDeserializer() != null) { + return CompatibilityDecision.requiresMigration( new EitherSerializer<>( - leftStrategy.getFallbackDeserializer(), - rightStrategy.getFallbackDeserializer())); + leftStrategy.getConvertDeserializer(), + rightStrategy.getConvertDeserializer())); } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } else { - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } } 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 2efdb2c815493..e325a3a248441 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/PojoSerializer.java @@ -33,8 +33,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.GenericTypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.TypeSerializerUtil; @@ -70,7 +70,7 @@ public final class PojoSerializer extends TypeSerializer { * handled with the {@link #readObject(ObjectInputStream)} and {@link #writeObject(ObjectOutputStream)} * methods. * - *

These may be reconfigured in {@link #getMigrationStrategy(TypeSerializerConfigSnapshot)}. + *

These may be reconfigured in {@link #ensureCompatibility(TypeSerializerConfigSnapshot)}. */ private transient Field[] fields; private TypeSerializer[] fieldSerializers; @@ -80,7 +80,7 @@ public final class PojoSerializer extends TypeSerializer { * Registered subclasses and their serializers. * Each subclass to their registered class tag is maintained as a separate map ordered by the class tag. * - *

These may be reconfigured in {@link #getMigrationStrategy(TypeSerializerConfigSnapshot)}. + *

These may be reconfigured in {@link #ensureCompatibility(TypeSerializerConfigSnapshot)}. */ private LinkedHashMap, Integer> registeredClasses; private TypeSerializer[] registeredSerializers; @@ -89,7 +89,7 @@ public final class PojoSerializer extends TypeSerializer { * Cache of non-registered subclasses to their serializers, created on-the-fly. * *

This cache is persisted and will be repopulated with reconfigured serializers - * in {@link #getMigrationStrategy(TypeSerializerConfigSnapshot)}. + * in {@link #ensureCompatibility(TypeSerializerConfigSnapshot)}. */ private transient HashMap, TypeSerializer> subclassSerializerCache; @@ -100,7 +100,7 @@ public final class PojoSerializer extends TypeSerializer { * *

Nested serializers created using this will have the most up-to-date configuration, * and can be resolved for backwards compatibility with previous configuration - * snapshots in {@link #getMigrationStrategy(TypeSerializerConfigSnapshot)}. + * snapshots in {@link #ensureCompatibility(TypeSerializerConfigSnapshot)}. */ private final ExecutionConfig executionConfig; @@ -554,14 +554,14 @@ public PojoSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - public MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof PojoSerializerConfigSnapshot) { final PojoSerializerConfigSnapshot config = (PojoSerializerConfigSnapshot) configSnapshot; if (clazz.equals(config.getTypeClass())) { if (this.numFields == config.getFieldToSerializerConfigSnapshot().size()) { - MigrationStrategy strategy; + CompatibilityDecision strategy; // ----------- check field order and migration requirement of field serializers ----------- @@ -581,12 +581,12 @@ public MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot co strategy = fieldSerializers[fieldIndex].getMigrationStrategyFor(fieldToConfigSnapshotEntry.getValue()); if (strategy.requireMigration()) { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } else { reorderedFieldSerializers[i] = fieldSerializers[fieldIndex]; } } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } i++; @@ -619,7 +619,7 @@ public MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot co // check migration requirement of subclass serializer strategy = reorderedRegisteredSubclassSerializers[i].getMigrationStrategyFor(previousRegisteredSerializerConfig); if (strategy.requireMigration()) { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } i++; @@ -639,7 +639,7 @@ public MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot co strategy = cachedSerializer.getMigrationStrategyFor(previousCachedEntry.getValue()); if (strategy.requireMigration()) { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } else { rebuiltCache.put(previousCachedEntry.getKey(), cachedSerializer); } @@ -656,12 +656,12 @@ public MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot co this.subclassSerializerCache = rebuiltCache; - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } } - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } public static final class PojoSerializerConfigSnapshot extends GenericTypeSerializerConfigSnapshot { 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 1ffd9fc8dabbb..f2cabab5ef739 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/RowSerializer.java @@ -18,8 +18,8 @@ package org.apache.flink.api.java.typeutils.runtime; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.TypeSerializerUtil; @@ -257,7 +257,7 @@ public RowSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof RowSerializerConfigSnapshot) { TypeSerializerConfigSnapshot[] fieldSerializerConfigSnapshots = ((RowSerializerConfigSnapshot) configSnapshot).getNestedSerializerConfigSnapshots(); @@ -266,30 +266,30 @@ protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapsh boolean requireMigration = false; TypeSerializer[] fallbackFieldSerializers = new TypeSerializer[fieldSerializers.length]; - MigrationStrategy strategy; + CompatibilityDecision strategy; for (int i = 0; i < fieldSerializers.length; i++) { strategy = fieldSerializers[i].getMigrationStrategyFor(fieldSerializerConfigSnapshots[i]); if (strategy.requireMigration()) { requireMigration = true; - if (strategy.getFallbackDeserializer() == null) { + if (strategy.getConvertDeserializer() == null) { // one of the field serializers cannot provide a fallback deserializer - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } else { - fallbackFieldSerializers[i] = strategy.getFallbackDeserializer(); + fallbackFieldSerializers[i] = strategy.getConvertDeserializer(); } } } if (requireMigration) { - return MigrationStrategy.migrateWithFallbackDeserializer(new RowSerializer(fallbackFieldSerializers)); + return CompatibilityDecision.requiresMigration(new RowSerializer(fallbackFieldSerializers)); } else { - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } } - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } public static final class RowSerializerConfigSnapshot extends CompositeTypeSerializerConfigSnapshot { 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 ca5c23c4dcada..4e4e114a57323 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 @@ -19,7 +19,7 @@ package org.apache.flink.api.java.typeutils.runtime; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.TypeSerializerUtil; @@ -132,7 +132,7 @@ public TupleSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof TupleSerializerConfigSnapshot) { final TupleSerializerConfigSnapshot config = (TupleSerializerConfigSnapshot) configSnapshot; @@ -142,19 +142,19 @@ protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot if (fieldSerializerConfigSnapshots.length == fieldSerializers.length) { - MigrationStrategy strategy; + CompatibilityDecision strategy; for (int i = 0; i < fieldSerializers.length; i++) { strategy = fieldSerializers[i].getMigrationStrategyFor(fieldSerializerConfigSnapshots[i]); if (strategy.requireMigration()) { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } } - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } 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 4ca8bcc595eee..a55714b52149d 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/ValueSerializer.java @@ -23,7 +23,7 @@ import java.util.LinkedHashMap; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; @@ -182,18 +182,18 @@ public ValueSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof ValueSerializerConfigSnapshot) { final ValueSerializerConfigSnapshot config = (ValueSerializerConfigSnapshot) configSnapshot; if (type.equals(config.getTypeClass())) { // currently, simply checking the type of the value class is sufficient; // in the future, if there are more Kryo registrations, we should try to resolve that - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } public static class ValueSerializerConfigSnapshot extends KryoRegistrationSerializerConfigSnapshot { 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 20f10bbc069e6..17a1d6aec7b40 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 @@ -28,7 +28,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.java.typeutils.runtime.DataInputViewStream; @@ -379,7 +379,7 @@ public KryoSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof KryoSerializerConfigSnapshot) { final KryoSerializerConfigSnapshot config = (KryoSerializerConfigSnapshot) configSnapshot; @@ -399,18 +399,18 @@ protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot "proper serializer, because its previous serializer cannot be loaded or is no " + "longer valid but a new serializer is not available", reconfiguredRegistrationEntry.getKey()); - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } // there's actually no way to tell if new Kryo serializers are compatible with // the previous ones they overwrite; we can only signal compatibly and hope for the best this.kryoRegistrations = reconfiguredRegistrations; - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } public static final class KryoSerializerConfigSnapshot extends KryoRegistrationSerializerConfigSnapshot { 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 398a53fef4c8b..4ff0f0c48a0ca 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 @@ -114,7 +114,7 @@ public void testSnapshotConfigurationAndReconfigure() throws Exception { new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } - MigrationStrategy strategy = getSerializer().getMigrationStrategyFor(restoredConfig); + CompatibilityDecision strategy = getSerializer().getMigrationStrategyFor(restoredConfig); assertFalse(strategy.requireMigration()); // also verify that the serializer's reconfigure implementation detects incompatibility diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotTest.java index 6ace128185232..be92476f969b7 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotTest.java @@ -39,7 +39,6 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; /** * Unit tests related to {@link TypeSerializerConfigSnapshot}. @@ -131,12 +130,12 @@ public void testMigrationStrategyWithForwardCompatibleMarkerConfig() { TypeSerializer mockSerializer = spy(TypeSerializer.class); mockSerializer.getMigrationStrategyFor(ForwardCompatibleSerializationFormatConfig.INSTANCE); - verify(mockSerializer, never()).getMigrationStrategy(any(TypeSerializerConfigSnapshot.class)); + verify(mockSerializer, never()).ensureCompatibility(any(TypeSerializerConfigSnapshot.class)); // make sure that is actually is called if its not the special marker TypeSerializerConfigSnapshot nonForwardCompatibleConfig = new TestConfigSnapshot(123, "foobar"); mockSerializer.getMigrationStrategyFor(nonForwardCompatibleConfig); - verify(mockSerializer, times(1)).getMigrationStrategy(nonForwardCompatibleConfig); + verify(mockSerializer, times(1)).ensureCompatibility(nonForwardCompatibleConfig); } public static class TestConfigSnapshot extends TypeSerializerConfigSnapshot { 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 4236dbd8c097a..bd1d774613613 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/base/EnumSerializerTest.java @@ -18,7 +18,7 @@ package org.apache.flink.api.common.typeutils.base; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.SerializerTestInstance; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.TypeSerializerUtil; @@ -72,7 +72,7 @@ public void testReconfiguration() { assertEquals(PublicEnum.PAULA.ordinal(), serializer.getValueToOrdinal().get(PublicEnum.PAULA).intValue()); // reconfigure and verify compatibility - MigrationStrategy strategy = serializer.getMigrationStrategyFor( + CompatibilityDecision strategy = serializer.getMigrationStrategyFor( new EnumSerializer.EnumSerializerConfigSnapshot<>(PublicEnum.class, mockPreviousOrder)); assertFalse(strategy.requireMigration()); @@ -106,7 +106,7 @@ public void testConfigurationSnapshotSerialization() throws Exception { new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } - MigrationStrategy strategy = serializer.getMigrationStrategyFor(restoredConfig); + CompatibilityDecision strategy = serializer.getMigrationStrategyFor(restoredConfig); assertFalse(strategy.requireMigration()); 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 - MigrationStrategy strategy = serializer.getMigrationStrategyFor( + CompatibilityDecision strategy = serializer.getMigrationStrategyFor( new EnumSerializer.EnumSerializerConfigSnapshot<>(PublicEnum.class, mockPreviousOrder)); assertFalse(strategy.requireMigration()); 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 fbb19bcf7f31b..9b9159b7bac26 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 @@ -31,9 +31,9 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.ForwardCompatibleSerializationFormatConfig; import org.apache.flink.api.common.typeutils.SerializerTestBase; -import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.CompositeType.FlatFieldDescriptor; @@ -300,7 +300,7 @@ public void testReconfigureWithDifferentPojoType() throws Exception { new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } - MigrationStrategy strategy = pojoSerializer2.getMigrationStrategy(pojoSerializerConfigSnapshot); + CompatibilityDecision strategy = pojoSerializer2.ensureCompatibility(pojoSerializerConfigSnapshot); assertTrue(strategy.requireMigration()); } @@ -340,7 +340,7 @@ public void testReconfigureDifferentSubclassRegistrationOrder() throws Exception new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } - MigrationStrategy result = pojoSerializer.getMigrationStrategy(pojoSerializerConfigSnapshot); + CompatibilityDecision result = pojoSerializer.ensureCompatibility(pojoSerializerConfigSnapshot); assertTrue(!result.requireMigration()); // reconfigure - check reconfiguration result and that registration ids remains the same @@ -384,7 +384,7 @@ public void testReconfigureRepopulateNonregisteredSubclassSerializerCache() thro } // reconfigure - check reconfiguration result and that subclass serializer cache is repopulated - MigrationStrategy strategy = pojoSerializer.getMigrationStrategyFor(pojoSerializerConfigSnapshot); + CompatibilityDecision strategy = pojoSerializer.getMigrationStrategyFor(pojoSerializerConfigSnapshot); assertFalse(strategy.requireMigration()); assertEquals(2, pojoSerializer.getSubclassSerializerCache().size()); assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassA.class)); @@ -446,7 +446,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 - MigrationStrategy strategy = pojoSerializer.getMigrationStrategyFor(pojoSerializerConfigSnapshot); + CompatibilityDecision strategy = pojoSerializer.getMigrationStrategyFor(pojoSerializerConfigSnapshot); assertFalse(strategy.requireMigration()); assertEquals(2, pojoSerializer.getSubclassSerializerCache().size()); assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassA.class)); @@ -494,7 +494,7 @@ public void testReconfigureWithDifferentFieldOrder() throws Exception { new HashMap, TypeSerializerConfigSnapshot>()); // empty; irrelevant for this test // reconfigure - check reconfiguration result and that fields are reordered to the previous order - MigrationStrategy strategy = pojoSerializer.getMigrationStrategyFor(mockPreviousConfigSnapshot); + CompatibilityDecision strategy = pojoSerializer.getMigrationStrategyFor(mockPreviousConfigSnapshot); assertFalse(strategy.requireMigration()); int i = 0; for (Field field : mockOriginalFieldOrder) { diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerMigrationTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerMigrationTest.java index b1cea46efe0db..83c82f60fafcf 100644 --- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerMigrationTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerMigrationTest.java @@ -23,7 +23,7 @@ import com.esotericsoftware.kryo.io.Input; import com.esotericsoftware.kryo.io.Output; import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.TypeSerializerUtil; import org.apache.flink.core.memory.DataInputViewStreamWrapper; @@ -65,7 +65,7 @@ public void testMigrationStrategyWithDifferentKryoType() throws Exception { new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } - MigrationStrategy strategy = kryoSerializerForB.getMigrationStrategy(kryoSerializerConfigSnapshot); + CompatibilityDecision strategy = kryoSerializerForB.ensureCompatibility(kryoSerializerConfigSnapshot); assertTrue(strategy.requireMigration()); } @@ -109,7 +109,7 @@ public void testMigrationStrategyForDifferentRegistrationOrder() throws Exceptio } // reconfigure - check reconfiguration result and that registration id remains the same - MigrationStrategy strategy = kryoSerializer.getMigrationStrategyFor(kryoSerializerConfigSnapshot); + CompatibilityDecision strategy = kryoSerializer.getMigrationStrategyFor(kryoSerializerConfigSnapshot); assertFalse(strategy.requireMigration()); assertEquals(testClassId, kryoSerializer.getKryo().getRegistration(TestClass.class).getId()); assertEquals(testClassAId, kryoSerializer.getKryo().getRegistration(TestClassA.class).getId()); 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 6024ad5d96f08..9dd093d962f84 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 @@ -26,7 +26,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.java.typeutils.TypeExtractor; @@ -395,7 +395,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { throw new UnsupportedOperationException(); } } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java index 3ad3f03278920..c548d214dd7e0 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java @@ -19,7 +19,7 @@ package org.apache.flink.cep; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; @@ -203,7 +203,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { throw new UnsupportedOperationException("This serializer is not registered for managed state."); } } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java index 7d629c9f8c753..cc719f1bfb04d 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java @@ -21,7 +21,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.base.CollectionSerializerConfigSnapshot; @@ -507,25 +507,25 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof CollectionSerializerConfigSnapshot) { - MigrationStrategy strategy = elementSerializer.getMigrationStrategyFor( + CompatibilityDecision strategy = elementSerializer.getMigrationStrategyFor( ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); if (strategy.requireMigration()) { - if (strategy.getFallbackDeserializer() != null) { - return MigrationStrategy.migrateWithFallbackDeserializer( + if (strategy.getConvertDeserializer() != null) { + return CompatibilityDecision.requiresMigration( new PriorityQueueSerializer<>( - strategy.getFallbackDeserializer(), + strategy.getConvertDeserializer(), factory)); } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } else { - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } } 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 dd319d34ec855..1e849534efa12 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 @@ -18,7 +18,7 @@ package org.apache.flink.table.runtime.types -import org.apache.flink.api.common.typeutils.{CompositeTypeSerializerConfigSnapshot, MigrationStrategy, TypeSerializer, TypeSerializerConfigSnapshot} +import org.apache.flink.api.common.typeutils.{CompositeTypeSerializerConfigSnapshot, CompatibilityDecision, TypeSerializer, TypeSerializerConfigSnapshot} import org.apache.flink.core.memory.{DataInputView, DataOutputView} import org.apache.flink.types.Row @@ -81,8 +81,8 @@ class CRowSerializer(val rowSerializer: TypeSerializer[Row]) extends TypeSeriali rowSerializer.snapshotConfiguration()) } - override protected def getMigrationStrategy( - configSnapshot: TypeSerializerConfigSnapshot): MigrationStrategy[CRow] = { + override protected def ensureCompatibility( + configSnapshot: TypeSerializerConfigSnapshot): CompatibilityDecision[CRow] = { configSnapshot match { case crowSerializerConfigSnapshot: CRowSerializer.CRowSerializerConfigSnapshot => @@ -90,18 +90,18 @@ class CRowSerializer(val rowSerializer: TypeSerializer[Row]) extends TypeSeriali crowSerializerConfigSnapshot.getSingleNestedSerializerConfigSnapshot) if (strategy.requireMigration()) { - if (strategy.getFallbackDeserializer != null) { - MigrationStrategy.migrateWithFallbackDeserializer( - new CRowSerializer(strategy.getFallbackDeserializer) + if (strategy.getConvertDeserializer != null) { + CompatibilityDecision.requiresMigration( + new CRowSerializer(strategy.getConvertDeserializer) ) } else { - MigrationStrategy.migrate() + CompatibilityDecision.requiresMigration(null) } } else { - MigrationStrategy.noMigration() + CompatibilityDecision.compatible() } - case _ => MigrationStrategy.migrate() + case _ => CompatibilityDecision.requiresMigration(null) } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java index 50c340d4a5270..866a1e3fc2fb3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java +++ b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java @@ -18,7 +18,7 @@ package org.apache.flink.migration; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.ParameterlessTypeSerializerConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; @@ -110,8 +110,8 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { - return MigrationStrategy.noMigration(); + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + return CompatibilityDecision.compatible(); } @Override 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 c0ec116ce6beb..29734c357c4c3 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/ArrayListSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/ArrayListSerializer.java @@ -17,7 +17,7 @@ */ package org.apache.flink.runtime.state; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.base.CollectionSerializerConfigSnapshot; @@ -146,23 +146,23 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof CollectionSerializerConfigSnapshot) { - MigrationStrategy strategy = elementSerializer.getMigrationStrategyFor( + CompatibilityDecision strategy = elementSerializer.getMigrationStrategyFor( ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); if (strategy.requireMigration()) { - if (strategy.getFallbackDeserializer() != null) { - return MigrationStrategy.migrateWithFallbackDeserializer( - new ArrayListSerializer<>(strategy.getFallbackDeserializer())); + if (strategy.getConvertDeserializer() != null) { + return CompatibilityDecision.requiresMigration( + new ArrayListSerializer<>(strategy.getConvertDeserializer())); } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } else { - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/HashMapSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/HashMapSerializer.java index 0377ddb8f869b..9767de405fa92 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/HashMapSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/HashMapSerializer.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.state; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.base.MapSerializerConfigSnapshot; @@ -208,28 +208,28 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof MapSerializerConfigSnapshot) { TypeSerializerConfigSnapshot[] keyValueSerializerConfigSnapshots = ((MapSerializerConfigSnapshot) configSnapshot).getNestedSerializerConfigSnapshots(); - MigrationStrategy keyStrategy = keySerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[0]); - MigrationStrategy valueStrategy = valueSerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[1]); + CompatibilityDecision keyStrategy = keySerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[0]); + CompatibilityDecision valueStrategy = valueSerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[1]); if (keyStrategy.requireMigration() || valueStrategy.requireMigration()) { - if (keyStrategy.getFallbackDeserializer() != null && valueStrategy.getFallbackDeserializer() != null) { - return MigrationStrategy.migrateWithFallbackDeserializer( + if (keyStrategy.getConvertDeserializer() != null && valueStrategy.getConvertDeserializer() != null) { + return CompatibilityDecision.requiresMigration( new HashMapSerializer<>( - keyStrategy.getFallbackDeserializer(), - valueStrategy.getFallbackDeserializer())); + keyStrategy.getConvertDeserializer(), + valueStrategy.getConvertDeserializer())); } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } else { - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } } 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 467c1aec2380a..a389c4f7ef7f8 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 @@ -31,7 +31,7 @@ import java.util.List; /** - * Serialization proxy for all meta data in keyed state backends. In the future we might also migrate the actual state + * Serialization proxy for all meta data in keyed state backends. In the future we might also requiresMigration the actual state * serialization logic here. */ public class KeyedBackendSerializationProxy extends VersionedIOReadableWritable { 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 476aaac931546..91d7aab10f5db 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 @@ -29,7 +29,7 @@ import java.util.List; /** - * Serialization proxy for all meta data in operator state backends. In the future we might also migrate the actual state + * Serialization proxy for all meta data in operator state backends. In the future we might also requiresMigration the actual state * serialization logic here. */ public class OperatorBackendSerializationProxy extends VersionedIOReadableWritable { 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 ee80878f0ac07..6ab228191c7fa 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListSerializer.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntListSerializer.java @@ -21,7 +21,7 @@ import java.io.IOException; import java.util.Arrays; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; @@ -136,7 +136,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision 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 40f15bef2affb..e523eb6ba6265 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairSerializer.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/IntPairSerializer.java @@ -21,7 +21,7 @@ import java.io.IOException; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; @@ -145,7 +145,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision 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 0dca9ec5a0874..06909f78ec45c 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairSerializer.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/types/StringPairSerializer.java @@ -20,7 +20,7 @@ import java.io.IOException; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; @@ -113,7 +113,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { throw new UnsupportedOperationException(); } } 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 987413750024a..a381c2b8b374b 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/state/heap/CopyOnWriteStateTableTest.java @@ -19,7 +19,7 @@ package org.apache.flink.runtime.state.heap; import org.apache.flink.api.common.state.StateDescriptor; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; 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; @@ -658,7 +658,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision 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 ca390eba5020f..3c0db721bb0bf 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/recordutils/RecordSerializer.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/testutils/recordutils/RecordSerializer.java @@ -21,7 +21,7 @@ import java.io.IOException; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; @@ -150,7 +150,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { throw new UnsupportedOperationException(); } } 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 61fc5963fcba8..146ce0e67bbb5 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 @@ -18,7 +18,7 @@ package org.apache.flink.api.scala.typeutils import org.apache.flink.annotation.Internal -import org.apache.flink.api.common.typeutils.{MigrationStrategy, TypeSerializer, TypeSerializerConfigSnapshot} +import org.apache.flink.api.common.typeutils.{CompatibilityDecision, TypeSerializer, TypeSerializerConfigSnapshot} import org.apache.flink.api.java.typeutils.runtime.EitherSerializerConfigSnapshot import org.apache.flink.core.memory.{DataInputView, DataOutputView} @@ -116,8 +116,8 @@ class EitherSerializer[A, B, T <: Either[A, B]]( rightSerializer.snapshotConfiguration()) } - override protected def getMigrationStrategy( - configSnapshot: TypeSerializerConfigSnapshot): MigrationStrategy[T] = { + override protected def ensureCompatibility( + configSnapshot: TypeSerializerConfigSnapshot): CompatibilityDecision[T] = { configSnapshot match { case eitherSerializerConfig: EitherSerializerConfigSnapshot => @@ -128,24 +128,24 @@ class EitherSerializer[A, B, T <: Either[A, B]]( val rightStrategy = rightSerializer.getMigrationStrategyFor(leftRightConfigs(1)) if (leftStrategy.requireMigration || rightStrategy.requireMigration) { - if (leftStrategy.getFallbackDeserializer != null - && rightStrategy.getFallbackDeserializer != null) { + if (leftStrategy.getConvertDeserializer != null + && rightStrategy.getConvertDeserializer != null) { - MigrationStrategy.migrateWithFallbackDeserializer( + CompatibilityDecision.requiresMigration( new EitherSerializer[A, B, T]( - leftStrategy.getFallbackDeserializer, - rightStrategy.getFallbackDeserializer + leftStrategy.getConvertDeserializer, + rightStrategy.getConvertDeserializer ) ) } else { - MigrationStrategy.migrate() + CompatibilityDecision.requiresMigration(null) } } else { - MigrationStrategy.noMigration() + CompatibilityDecision.compatible() } - case _ => MigrationStrategy.migrate() + case _ => CompatibilityDecision.requiresMigration(null) } } } 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 b5d4663051967..4af9b6d1ecd6d 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/EnumValueSerializer.scala @@ -20,7 +20,7 @@ package org.apache.flink.api.scala.typeutils import java.io.IOException import org.apache.flink.annotation.Internal -import org.apache.flink.api.common.typeutils.{MigrationStrategy, TypeSerializer, TypeSerializerConfigSnapshot} +import org.apache.flink.api.common.typeutils.{CompatibilityDecision, TypeSerializer, TypeSerializerConfigSnapshot} import org.apache.flink.api.common.typeutils.base.IntSerializer import org.apache.flink.api.java.typeutils.runtime.{DataInputViewStream, DataOutputViewStream} import org.apache.flink.core.memory.{DataInputView, DataOutputView} @@ -81,8 +81,8 @@ class EnumValueSerializer[E <: Enumeration](val enum: E) extends TypeSerializer[ enum.getClass.asInstanceOf[Class[E]]) } - override protected def getMigrationStrategy( - configSnapshot: TypeSerializerConfigSnapshot): MigrationStrategy[E#Value] = { + override protected def ensureCompatibility( + configSnapshot: TypeSerializerConfigSnapshot): CompatibilityDecision[E#Value] = { configSnapshot match { case enumSerializerConfigSnapshot: EnumValueSerializer.ScalaEnumSerializerConfigSnapshot[_] => @@ -95,16 +95,16 @@ class EnumValueSerializer[E <: Enumeration](val enum: E) extends TypeSerializer[ // and original constants must be in the exact same order if (currentEnumConstants(i) != enumSerializerConfigSnapshot.getEnumConstants(i)) { - MigrationStrategy.migrate() + CompatibilityDecision.requiresMigration(null) } } - MigrationStrategy.noMigration() + CompatibilityDecision.compatible() } else { - MigrationStrategy.migrate() + CompatibilityDecision.requiresMigration(null) } - case _ => MigrationStrategy.migrate() + case _ => CompatibilityDecision.requiresMigration(null) } } } 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 6048c46052987..0a37ea8c99a18 100644 --- a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala +++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/NothingSerializer.scala @@ -18,7 +18,7 @@ package org.apache.flink.api.scala.typeutils import org.apache.flink.annotation.Internal -import org.apache.flink.api.common.typeutils.{MigrationStrategy, TypeSerializer, TypeSerializerConfigSnapshot} +import org.apache.flink.api.common.typeutils.{CompatibilityDecision, TypeSerializer, TypeSerializerConfigSnapshot} import org.apache.flink.core.memory.{DataInputView, DataOutputView} /** @@ -59,8 +59,8 @@ class NothingSerializer extends TypeSerializer[Any] { override def snapshotConfiguration(): TypeSerializerConfigSnapshot = throw new RuntimeException("This must not be used. You encountered a bug.") - override protected def getMigrationStrategy( - configSnapshot: TypeSerializerConfigSnapshot): MigrationStrategy[Any] = + override protected def ensureCompatibility( + configSnapshot: TypeSerializerConfigSnapshot): CompatibilityDecision[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 8147a3921f2d2..10469a93b980f 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 @@ -104,25 +104,25 @@ class OptionSerializer[A](val elemSerializer: TypeSerializer[A]) new OptionSerializer.OptionSerializerConfigSnapshot(elemSerializer.snapshotConfiguration()) } - override protected def getMigrationStrategy( - configSnapshot: TypeSerializerConfigSnapshot): MigrationStrategy[Option[A]] = { + override protected def ensureCompatibility( + configSnapshot: TypeSerializerConfigSnapshot): CompatibilityDecision[Option[A]] = { configSnapshot match { case optionSerializerConfigSnapshot: OptionSerializer.OptionSerializerConfigSnapshot => val strategy = elemSerializer.getMigrationStrategyFor( optionSerializerConfigSnapshot.getSingleNestedSerializerConfigSnapshot) if (strategy.requireMigration()) { - if (strategy.getFallbackDeserializer != null) { - MigrationStrategy.migrateWithFallbackDeserializer( - new OptionSerializer[A](strategy.getFallbackDeserializer)) + if (strategy.getConvertDeserializer != null) { + CompatibilityDecision.requiresMigration( + new OptionSerializer[A](strategy.getConvertDeserializer)) } else { - MigrationStrategy.migrate() + CompatibilityDecision.requiresMigration(null) } } else { - MigrationStrategy.noMigration() + CompatibilityDecision.compatible() } - case _ => MigrationStrategy.migrate() + case _ => CompatibilityDecision.requiresMigration(null) } } } 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 2a2d4e8cb5dcc..e51ba112834bd 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 @@ -20,7 +20,7 @@ package org.apache.flink.api.scala.typeutils import java.io.ObjectInputStream import org.apache.flink.annotation.Internal -import org.apache.flink.api.common.typeutils.{MigrationStrategy, TypeSerializer, TypeSerializerConfigSnapshot} +import org.apache.flink.api.common.typeutils.{CompatibilityDecision, TypeSerializer, TypeSerializerConfigSnapshot} import org.apache.flink.core.memory.{DataInputView, DataOutputView} import scala.collection.generic.CanBuildFrom @@ -155,8 +155,8 @@ abstract class TraversableSerializer[T <: TraversableOnce[E], E]( throw new UnsupportedOperationException() } - override protected def getMigrationStrategy( - configSnapshot: TypeSerializerConfigSnapshot): MigrationStrategy[T] = { + override protected def ensureCompatibility( + configSnapshot: TypeSerializerConfigSnapshot): CompatibilityDecision[T] = { throw new UnsupportedOperationException() } } 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 1aa5ce8054cfd..0f0d4f16bba3c 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 @@ -110,8 +110,8 @@ class TrySerializer[A]( throwableSerializer.snapshotConfiguration()) } - override protected def getMigrationStrategy( - configSnapshot: TypeSerializerConfigSnapshot): MigrationStrategy[Try[A]] = { + override protected def ensureCompatibility( + configSnapshot: TypeSerializerConfigSnapshot): CompatibilityDecision[Try[A]] = { configSnapshot match { case trySerializerConfigSnapshot: TrySerializer.TrySerializerConfigSnapshot => @@ -124,12 +124,12 @@ class TrySerializer[A]( throwableSerializer.getMigrationStrategyFor(serializerConfigSnapshots(1)) if (elemStrategy.requireMigration() || throwableStrategy.requireMigration()) { - MigrationStrategy.migrate() + CompatibilityDecision.requiresMigration(null) } else { - MigrationStrategy.noMigration() + CompatibilityDecision.compatible() } - case _ => MigrationStrategy.migrate() + case _ => CompatibilityDecision.requiresMigration(null) } } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java index 93805d3a8d4e8..8de22a9ebed3f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java @@ -22,8 +22,8 @@ import java.io.IOException; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; @@ -219,24 +219,24 @@ public MultiplexingStreamRecordSerializerConfigSnapshot snapshotConfiguration() } @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof MultiplexingStreamRecordSerializerConfigSnapshot) { - MigrationStrategy strategy = typeSerializer.getMigrationStrategyFor( + CompatibilityDecision strategy = typeSerializer.getMigrationStrategyFor( ((MultiplexingStreamRecordSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); if (strategy.requireMigration()) { - if (strategy.getFallbackDeserializer() != null) { - return MigrationStrategy.migrateWithFallbackDeserializer( + if (strategy.getConvertDeserializer() != null) { + return CompatibilityDecision.requiresMigration( new MultiplexingStreamRecordSerializer<>( - strategy.getFallbackDeserializer())); + strategy.getConvertDeserializer())); } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } else { - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java index 578ccb8b11179..7a782c6c4d266 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java @@ -20,8 +20,8 @@ import java.io.IOException; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; @@ -158,23 +158,23 @@ public StreamRecordSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof StreamRecordSerializerConfigSnapshot) { - MigrationStrategy strategy = typeSerializer.getMigrationStrategyFor( + CompatibilityDecision strategy = typeSerializer.getMigrationStrategyFor( ((StreamRecordSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); if (strategy.requireMigration()) { - if (strategy.getFallbackDeserializer() != null) { - return MigrationStrategy.migrateWithFallbackDeserializer( - new StreamRecordSerializer<>(strategy.getFallbackDeserializer())); + if (strategy.getConvertDeserializer() != null) { + return CompatibilityDecision.requiresMigration( + new StreamRecordSerializer<>(strategy.getConvertDeserializer())); } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } else { - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } 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 bc16a03609f7a..80822408a72a2 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 @@ -29,7 +29,7 @@ import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.java.functions.KeySelector; @@ -556,7 +556,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision> 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/api/operators/InternalTimer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimer.java index c1fdde5de82d2..9d09753e99e4d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimer.java @@ -19,7 +19,7 @@ import java.io.IOException; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.MigrationStrategy; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.api.common.typeutils.base.LongSerializer; @@ -193,7 +193,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy> getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision> 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 55b1ba5b95fe5..4005e4b263471 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,8 +22,8 @@ import java.io.IOException; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompatibilityDecision; import org.apache.flink.api.common.typeutils.CompositeTypeSerializerConfigSnapshot; -import org.apache.flink.api.common.typeutils.MigrationStrategy; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerConfigSnapshot; import org.apache.flink.core.memory.DataInputView; @@ -280,24 +280,24 @@ public StreamElementSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected MigrationStrategy getMigrationStrategy(TypeSerializerConfigSnapshot configSnapshot) { + protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof StreamElementSerializerConfigSnapshot) { - MigrationStrategy strategy = typeSerializer.getMigrationStrategyFor( + CompatibilityDecision strategy = typeSerializer.getMigrationStrategyFor( ((StreamElementSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); if (strategy.requireMigration()) { - if (strategy.getFallbackDeserializer() != null) { - return MigrationStrategy.migrateWithFallbackDeserializer( + if (strategy.getConvertDeserializer() != null) { + return CompatibilityDecision.requiresMigration( new StreamElementSerializer<>( - strategy.getFallbackDeserializer())); + strategy.getConvertDeserializer())); } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } else { - return MigrationStrategy.noMigration(); + return CompatibilityDecision.compatible(); } } else { - return MigrationStrategy.migrate(); + return CompatibilityDecision.requiresMigration(null); } } From 978aed0797c778d2a7b0ccbfd81a8221514d05f5 Mon Sep 17 00:00:00 2001 From: "Tzu-Li (Gordon) Tai" Date: Sun, 7 May 2017 20:17:29 +0800 Subject: [PATCH 5/5] [FLINK-6425] Remove ForwardCompatibleSerializationFormatConfig shortcut --- .../typeutils/runtime/WritableSerializer.java | 2 +- .../state/RocksDBKeyedStateBackend.java | 4 +- ...rdCompatibleSerializationFormatConfig.java | 76 ------------------- .../api/common/typeutils/TypeSerializer.java | 45 +---------- .../TypeSerializerSerializationProxy.java | 2 +- .../common/typeutils/TypeSerializerUtil.java | 51 +++++-------- .../common/typeutils/base/EnumSerializer.java | 2 +- .../base/GenericArraySerializer.java | 4 +- .../common/typeutils/base/ListSerializer.java | 4 +- .../common/typeutils/base/MapSerializer.java | 6 +- .../typeutils/runtime/AvroSerializer.java | 2 +- .../runtime/CopyableValueSerializer.java | 2 +- .../typeutils/runtime/EitherSerializer.java | 6 +- .../typeutils/runtime/PojoSerializer.java | 11 ++- .../java/typeutils/runtime/RowSerializer.java | 4 +- .../runtime/TupleSerializerBase.java | 4 +- .../typeutils/runtime/ValueSerializer.java | 2 +- .../runtime/kryo/KryoSerializer.java | 2 +- .../common/typeutils/SerializerTestBase.java | 4 +- .../TypeSerializerConfigSnapshotTest.java | 49 ------------ .../typeutils/base/EnumSerializerTest.java | 6 +- .../typeutils/runtime/PojoSerializerTest.java | 22 ++++-- .../kryo/KryoSerializerMigrationTest.java | 2 +- .../java/io/CollectionInputFormatTest.java | 2 +- .../cep/NonDuplicatingTypeSerializer.java | 2 +- .../AbstractKeyedCEPPatternOperator.java | 4 +- .../table/runtime/types/CRowSerializer.scala | 4 +- .../MigrationNamespaceSerializerProxy.java | 2 +- .../runtime/state/ArrayListSerializer.java | 4 +- .../runtime/state/HashMapSerializer.java | 6 +- .../testutils/types/IntListSerializer.java | 2 +- .../testutils/types/IntPairSerializer.java | 2 +- .../testutils/types/StringPairSerializer.java | 2 +- .../state/heap/CopyOnWriteStateTableTest.java | 2 +- .../recordutils/RecordSerializer.java | 2 +- .../scala/typeutils/EitherSerializer.scala | 6 +- .../scala/typeutils/EnumValueSerializer.scala | 2 +- .../scala/typeutils/NothingSerializer.scala | 2 +- .../scala/typeutils/OptionSerializer.scala | 4 +- .../typeutils/TraversableSerializer.scala | 2 +- .../api/scala/typeutils/TrySerializer.scala | 6 +- .../MultiplexingStreamRecordSerializer.java | 4 +- .../streamrecord/StreamRecordSerializer.java | 4 +- .../api/datastream/CoGroupedStreams.java | 2 +- .../api/operators/InternalTimer.java | 2 +- .../streamrecord/StreamElementSerializer.java | 4 +- 46 files changed, 107 insertions(+), 277 deletions(-) delete mode 100644 flink-core/src/main/java/org/apache/flink/api/common/typeutils/ForwardCompatibleSerializationFormatConfig.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 a3c28be7ea0bf..602c10882c2ff 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 @@ -165,7 +165,7 @@ public WritableSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof WritableSerializerConfigSnapshot && typeClass.equals(((WritableSerializerConfigSnapshot) configSnapshot).getTypeClass())) { diff --git a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java index ef255128052c4..2e7c71b64bce6 100644 --- a/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java +++ b/flink-contrib/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBKeyedStateBackend.java @@ -1522,7 +1522,7 @@ protected ColumnFamilyHandle getColumnFamily( if (restoredMetaInfo.getNamespaceSerializerConfigSnapshot() != null) { CompatibilityDecision namespaceCompatibilityDecision = newMetaInfo.getNamespaceSerializer() - .getMigrationStrategyFor(restoredMetaInfo.getNamespaceSerializerConfigSnapshot()); + .ensureCompatibility(restoredMetaInfo.getNamespaceSerializerConfigSnapshot()); TypeSerializer finalOldNamespaceSerializer; if (namespaceCompatibilityDecision.requireMigration()) { @@ -1542,7 +1542,7 @@ protected ColumnFamilyHandle getColumnFamily( if (restoredMetaInfo.getStateSerializerConfigSnapshot() != null) { CompatibilityDecision stateCompatibilityDecision = newMetaInfo.getStateSerializer() - .getMigrationStrategyFor(restoredMetaInfo.getStateSerializerConfigSnapshot()); + .ensureCompatibility(restoredMetaInfo.getStateSerializerConfigSnapshot()); TypeSerializer finalOldStateSerializer; if (stateCompatibilityDecision.requireMigration()) { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/ForwardCompatibleSerializationFormatConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/ForwardCompatibleSerializationFormatConfig.java deleted file mode 100644 index 4d81e31c503aa..0000000000000 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/ForwardCompatibleSerializationFormatConfig.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.flink.api.common.typeutils; - -import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; - -import java.io.IOException; - -/** - * This is a special {@link TypeSerializerConfigSnapshot} that serializers can return to serve - * as a marker to indicate that new serializers for the data written by this serializer does not - * need to be checked for compatibility. - */ -@PublicEvolving -public final class ForwardCompatibleSerializationFormatConfig extends TypeSerializerConfigSnapshot { - - /** Singleton instance. */ - public static final ForwardCompatibleSerializationFormatConfig INSTANCE = - new ForwardCompatibleSerializationFormatConfig(); - - @Override - public void write(DataOutputView out) throws IOException { - // nothing to write - } - - @Override - public void read(DataInputView in) throws IOException { - // nothing to read - } - - @Override - public int getSnapshotVersion() { - throw new UnsupportedOperationException( - "This is a ForwardCompatibleSerializationFormatConfig. No versioning required."); - } - - @Override - public int getVersion() { - throw new UnsupportedOperationException( - "This is a ForwardCompatibleSerializationFormatConfig. No versioning required."); - } - - /** - * This special configuration type does not require the default - * empty nullary constructor because it will never actually be serialized. - */ - private ForwardCompatibleSerializationFormatConfig() {} - - @Override - public boolean equals(Object obj) { - return obj instanceof ForwardCompatibleSerializationFormatConfig; - } - - @Override - public int hashCode() { - return getClass().hashCode(); - } -} 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 a52c975a55a50..5ed4e7458369e 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,7 +18,6 @@ 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; @@ -176,17 +175,7 @@ public abstract class TypeSerializer implements Serializable { * serializer was registered to, the returned configuration snapshot can be used to check with the new serializer * if any data migration needs to take place. * - *

Implementations can also return the singleton {@link ForwardCompatibleSerializationFormatConfig#INSTANCE} - * configuration if they guarantee forwards compatibility. For example, implementations that use serialization - * frameworks with built-in serialization compatibility, such as Thrift or - * Protobuf, is suitable for this usage pattern. By - * returning the {@link ForwardCompatibleSerializationFormatConfig#INSTANCE}, this informs Flink that when managed - * state serialized using this serializer is restored, there is no need to check for migration with the new - * serializer for the same state. In other words, new serializers are always assumed to be fully compatible for the - * serialized state. - * * @see TypeSerializerConfigSnapshot - * @see ForwardCompatibleSerializationFormatConfig * * @return snapshot of the serializer's current configuration. */ @@ -209,43 +198,11 @@ public abstract class TypeSerializer implements Serializable { * restored to read the old data, the provided convert deserializer can be used. * * - *

This method is guaranteed to only be invoked if the preceding serializer's configuration snapshot is not the - * singleton {@link ForwardCompatibleSerializationFormatConfig#INSTANCE} configuration. In such cases, Flink always - * assume that the migration strategy is {@link CompatibilityDecision#requiresMigration(TypeSerializer)}. - * * @see CompatibilityDecision * * @param configSnapshot configuration snapshot of a preceding serializer for the same managed state * * @return the result of the reconfiguration. */ - protected abstract CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot); - - /** - * Get the migration strategy to use this serializer based on the configuration snapshot of 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). - * - *

This method is not part of the public user-facing API, and cannot be overriden. External operations - * providing a configuration snapshot of preceding serializer can only do so through this method. - * - *

This method always assumes that the migration strategy is {@link CompatibilityDecision#compatible()} if - * the provided configuration snapshot is the singleton {@link ForwardCompatibleSerializationFormatConfig#INSTANCE}. - * Otherwise, the configuration snapshot is provided to the actual - * {@link #ensureCompatibility(TypeSerializerConfigSnapshot)} (TypeSerializerConfigSnapshot)} implementation. - * - * @param configSnapshot configuration snapshot of a preceding serializer for the same managed state - * - * @return the result of the reconfiguration. - */ - @Internal - public final CompatibilityDecision getMigrationStrategyFor(TypeSerializerConfigSnapshot configSnapshot) { - // reference equality is viable here, because the forward compatible - // marker config will always be explicitly restored with the singleton instance - if (configSnapshot != ForwardCompatibleSerializationFormatConfig.INSTANCE) { - return ensureCompatibility(configSnapshot); - } else { - return CompatibilityDecision.compatible(); - } - } + public abstract CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot); } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java index 04e02cd4c5057..f3d1a911673fb 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerSerializationProxy.java @@ -212,7 +212,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision 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/TypeSerializerUtil.java b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerUtil.java index 768d8d2b30070..e8dc15e62bb77 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerUtil.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/typeutils/TypeSerializerUtil.java @@ -162,20 +162,12 @@ static class TypeSerializerConfigSnapshotProxy extends VersionedIOReadableWritab public void write(DataOutputView out) throws IOException { super.write(out); - if (serializerConfigSnapshot == ForwardCompatibleSerializationFormatConfig.INSTANCE) { - // if the config is actually the special forward compatible config, - // just write a flag to indicate that and write nothing else - out.writeBoolean(false); - } else { - out.writeBoolean(true); - - // 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); - } + // 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") @@ -183,25 +175,20 @@ public void write(DataOutputView out) throws IOException { public void read(DataInputView in) throws IOException { super.read(in); - if (in.readBoolean()) { - 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); - } else { - // was a special forward compatible config; restore with the singleton instance - serializerConfigSnapshot = ForwardCompatibleSerializationFormatConfig.INSTANCE; + 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 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 bd2be6d2a52bf..f264e1f80a06a 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 @@ -179,7 +179,7 @@ public EnumSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision 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 8011567076f84..994b7f7a3ae5a 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 @@ -199,12 +199,12 @@ public GenericArraySerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof GenericArraySerializerConfigSnapshot) { final GenericArraySerializerConfigSnapshot config = (GenericArraySerializerConfigSnapshot) configSnapshot; if (componentClass.equals(config.getComponentClass())) { - CompatibilityDecision strategy = componentSerializer.getMigrationStrategyFor( + CompatibilityDecision strategy = componentSerializer.ensureCompatibility( config.getSingleNestedSerializerConfigSnapshot()); if (strategy.requireMigration()) { 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 04b071aa14fb3..34211422f80de 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,9 +179,9 @@ public CollectionSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof CollectionSerializerConfigSnapshot) { - CompatibilityDecision strategy = elementSerializer.getMigrationStrategyFor( + CompatibilityDecision strategy = elementSerializer.ensureCompatibility( ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); if (strategy.requireMigration()) { 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 e3bb6a72dae28..463e134aafef8 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,13 +207,13 @@ public MapSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof MapSerializerConfigSnapshot) { TypeSerializerConfigSnapshot[] keyValueSerializerConfigSnapshots = ((MapSerializerConfigSnapshot) configSnapshot).getNestedSerializerConfigSnapshots(); - CompatibilityDecision keyStrategy = keySerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[0]); - CompatibilityDecision valueStrategy = valueSerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[1]); + CompatibilityDecision keyStrategy = keySerializer.ensureCompatibility(keyValueSerializerConfigSnapshots[0]); + CompatibilityDecision valueStrategy = valueSerializer.ensureCompatibility(keyValueSerializerConfigSnapshots[1]); if (keyStrategy.requireMigration() || valueStrategy.requireMigration()) { if (keyStrategy.getConvertDeserializer() != null && valueStrategy.getConvertDeserializer() != null) { diff --git a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java index c1ee2eca21906..3d7242dece852 100644 --- a/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java +++ b/flink-core/src/main/java/org/apache/flink/api/java/typeutils/runtime/AvroSerializer.java @@ -225,7 +225,7 @@ public AvroSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof AvroSerializerConfigSnapshot) { final AvroSerializerConfigSnapshot config = (AvroSerializerConfigSnapshot) configSnapshot; 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 b4d69b6a215fc..d9385a9f6d582 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,7 +142,7 @@ public CopyableValueSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof CopyableValueSerializerConfigSnapshot && valueClass.equals(((CopyableValueSerializerConfigSnapshot) configSnapshot).getTypeClass())) { return CompatibilityDecision.compatible(); 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 26c11c30e08be..30417c22d2530 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 @@ -198,13 +198,13 @@ public EitherSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof EitherSerializerConfigSnapshot) { TypeSerializerConfigSnapshot[] leftRightSerializerConfigSnapshots = ((EitherSerializerConfigSnapshot) configSnapshot).getNestedSerializerConfigSnapshots(); - CompatibilityDecision leftStrategy = leftSerializer.getMigrationStrategyFor(leftRightSerializerConfigSnapshots[0]); - CompatibilityDecision rightStrategy = rightSerializer.getMigrationStrategyFor(leftRightSerializerConfigSnapshots[1]); + CompatibilityDecision leftStrategy = leftSerializer.ensureCompatibility(leftRightSerializerConfigSnapshots[0]); + CompatibilityDecision rightStrategy = rightSerializer.ensureCompatibility(leftRightSerializerConfigSnapshots[1]); if (leftStrategy.requireMigration() || rightStrategy.requireMigration()) { if (leftStrategy.getConvertDeserializer() != null && rightStrategy.getConvertDeserializer() != null) { 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 e325a3a248441..09805853e2a49 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 @@ -579,7 +579,7 @@ public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot if (fieldIndex != -1) { reorderedFields[i] = fieldToConfigSnapshotEntry.getKey(); - strategy = fieldSerializers[fieldIndex].getMigrationStrategyFor(fieldToConfigSnapshotEntry.getValue()); + strategy = fieldSerializers[fieldIndex].ensureCompatibility(fieldToConfigSnapshotEntry.getValue()); if (strategy.requireMigration()) { return CompatibilityDecision.requiresMigration(null); } else { @@ -617,7 +617,7 @@ public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot i = 0; for (TypeSerializerConfigSnapshot previousRegisteredSerializerConfig : previousRegistrations.values()) { // check migration requirement of subclass serializer - strategy = reorderedRegisteredSubclassSerializers[i].getMigrationStrategyFor(previousRegisteredSerializerConfig); + strategy = reorderedRegisteredSubclassSerializers[i].ensureCompatibility(previousRegisteredSerializerConfig); if (strategy.requireMigration()) { return CompatibilityDecision.requiresMigration(null); } @@ -637,7 +637,7 @@ public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot // check migration requirement of cached subclass serializer TypeSerializer cachedSerializer = createSubclassSerializer(previousCachedEntry.getKey()); - strategy = cachedSerializer.getMigrationStrategyFor(previousCachedEntry.getValue()); + strategy = cachedSerializer.ensureCompatibility(previousCachedEntry.getValue()); if (strategy.requireMigration()) { return CompatibilityDecision.requiresMigration(null); } else { @@ -1040,6 +1040,11 @@ Field[] getFields() { return fields; } + @VisibleForTesting + TypeSerializer[] getFieldSerializers() { + return fieldSerializers; + } + @VisibleForTesting LinkedHashMap, Integer> getRegisteredClasses() { return registeredClasses; 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 f2cabab5ef739..58b291b280c1c 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 @@ -257,7 +257,7 @@ public RowSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof RowSerializerConfigSnapshot) { TypeSerializerConfigSnapshot[] fieldSerializerConfigSnapshots = ((RowSerializerConfigSnapshot) configSnapshot).getNestedSerializerConfigSnapshots(); @@ -268,7 +268,7 @@ protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSna CompatibilityDecision strategy; for (int i = 0; i < fieldSerializers.length; i++) { - strategy = fieldSerializers[i].getMigrationStrategyFor(fieldSerializerConfigSnapshots[i]); + strategy = fieldSerializers[i].ensureCompatibility(fieldSerializerConfigSnapshots[i]); if (strategy.requireMigration()) { requireMigration = true; 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 4e4e114a57323..d76d85581d9bc 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 @@ -132,7 +132,7 @@ public TupleSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof TupleSerializerConfigSnapshot) { final TupleSerializerConfigSnapshot config = (TupleSerializerConfigSnapshot) configSnapshot; @@ -144,7 +144,7 @@ protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnaps CompatibilityDecision strategy; for (int i = 0; i < fieldSerializers.length; i++) { - strategy = fieldSerializers[i].getMigrationStrategyFor(fieldSerializerConfigSnapshots[i]); + strategy = fieldSerializers[i].ensureCompatibility(fieldSerializerConfigSnapshots[i]); if (strategy.requireMigration()) { return CompatibilityDecision.requiresMigration(null); } 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 a55714b52149d..41c9457648b5b 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 - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision 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 17a1d6aec7b40..c506d804c394f 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 @@ -379,7 +379,7 @@ public KryoSerializerConfigSnapshot snapshotConfiguration() { @SuppressWarnings("unchecked") @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision 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 4ff0f0c48a0ca..71b22574f0b1d 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 @@ -114,11 +114,11 @@ public void testSnapshotConfigurationAndReconfigure() throws Exception { new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } - CompatibilityDecision strategy = getSerializer().getMigrationStrategyFor(restoredConfig); + CompatibilityDecision strategy = getSerializer().ensureCompatibility(restoredConfig); assertFalse(strategy.requireMigration()); // also verify that the serializer's reconfigure implementation detects incompatibility - strategy = getSerializer().getMigrationStrategyFor(new TestIncompatibleSerializerConfigSnapshot()); + strategy = getSerializer().ensureCompatibility(new TestIncompatibleSerializerConfigSnapshot()); assertTrue(strategy.requireMigration()); } diff --git a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotTest.java b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotTest.java index be92476f969b7..0783bb604e401 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/typeutils/TypeSerializerConfigSnapshotTest.java @@ -31,14 +31,7 @@ import java.net.URLClassLoader; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; /** * Unit tests related to {@link TypeSerializerConfigSnapshot}. @@ -96,48 +89,6 @@ public void testFailsWhenConfigurationSnapshotClassNotFound() throws Exception { } } - /** - * Tests that serializing and then deserializing the special marker config - * {@link ForwardCompatibleSerializationFormatConfig#INSTANCE} always - * restores the singleton instance. - */ - @Test - public void testSerializeForwardCompatibleMarkerConfig() throws Exception { - byte[] serializedConfig; - try (ByteArrayOutputStream out = new ByteArrayOutputStream()) { - TypeSerializerUtil.writeSerializerConfigSnapshot( - new DataOutputViewStreamWrapper(out), ForwardCompatibleSerializationFormatConfig.INSTANCE); - serializedConfig = out.toByteArray(); - } - - TypeSerializerConfigSnapshot restoredConfig; - try (ByteArrayInputStream in = new ByteArrayInputStream(serializedConfig)) { - restoredConfig = TypeSerializerUtil.readSerializerConfigSnapshot( - new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); - } - - // reference equality to the singleton instance - assertTrue(restoredConfig == ForwardCompatibleSerializationFormatConfig.INSTANCE); - } - - /** - * Verifies that the actual reconfigure method is never invoked if the - * provided configuration snapshot is the special singleton marker config - * {@link ForwardCompatibleSerializationFormatConfig#INSTANCE}. - */ - @Test - public void testMigrationStrategyWithForwardCompatibleMarkerConfig() { - TypeSerializer mockSerializer = spy(TypeSerializer.class); - - mockSerializer.getMigrationStrategyFor(ForwardCompatibleSerializationFormatConfig.INSTANCE); - verify(mockSerializer, never()).ensureCompatibility(any(TypeSerializerConfigSnapshot.class)); - - // make sure that is actually is called if its not the special marker - TypeSerializerConfigSnapshot nonForwardCompatibleConfig = new TestConfigSnapshot(123, "foobar"); - mockSerializer.getMigrationStrategyFor(nonForwardCompatibleConfig); - verify(mockSerializer, times(1)).ensureCompatibility(nonForwardCompatibleConfig); - } - public static class TestConfigSnapshot extends TypeSerializerConfigSnapshot { static final int VERSION = 1; 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 bd1d774613613..3b7e0c5173661 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 - CompatibilityDecision strategy = serializer.getMigrationStrategyFor( + CompatibilityDecision strategy = serializer.ensureCompatibility( new EnumSerializer.EnumSerializerConfigSnapshot<>(PublicEnum.class, mockPreviousOrder)); assertFalse(strategy.requireMigration()); @@ -106,7 +106,7 @@ public void testConfigurationSnapshotSerialization() throws Exception { new DataInputViewStreamWrapper(in), Thread.currentThread().getContextClassLoader()); } - CompatibilityDecision strategy = serializer.getMigrationStrategyFor(restoredConfig); + CompatibilityDecision strategy = serializer.ensureCompatibility(restoredConfig); assertFalse(strategy.requireMigration()); 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 - CompatibilityDecision strategy = serializer.getMigrationStrategyFor( + CompatibilityDecision strategy = serializer.ensureCompatibility( new EnumSerializer.EnumSerializerConfigSnapshot<>(PublicEnum.class, mockPreviousOrder)); assertFalse(strategy.requireMigration()); 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 9b9159b7bac26..551d8de871ede 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 @@ -32,7 +32,6 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.CompatibilityDecision; -import org.apache.flink.api.common.typeutils.ForwardCompatibleSerializationFormatConfig; import org.apache.flink.api.common.typeutils.SerializerTestBase; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -384,7 +383,7 @@ public void testReconfigureRepopulateNonregisteredSubclassSerializerCache() thro } // reconfigure - check reconfiguration result and that subclass serializer cache is repopulated - CompatibilityDecision strategy = pojoSerializer.getMigrationStrategyFor(pojoSerializerConfigSnapshot); + CompatibilityDecision strategy = pojoSerializer.ensureCompatibility(pojoSerializerConfigSnapshot); assertFalse(strategy.requireMigration()); assertEquals(2, pojoSerializer.getSubclassSerializerCache().size()); assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassA.class)); @@ -446,7 +445,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 - CompatibilityDecision strategy = pojoSerializer.getMigrationStrategyFor(pojoSerializerConfigSnapshot); + CompatibilityDecision strategy = pojoSerializer.ensureCompatibility(pojoSerializerConfigSnapshot); assertFalse(strategy.requireMigration()); assertEquals(2, pojoSerializer.getSubclassSerializerCache().size()); assertTrue(pojoSerializer.getSubclassSerializerCache().containsKey(SubTestUserClassA.class)); @@ -470,12 +469,17 @@ public void testReconfigureWithDifferentFieldOrder() throws Exception { TestUserClass.class.getField("dumm5"), }; + // creating this serializer just for generating config snapshots of the field serializers + PojoSerializer ser = (PojoSerializer) type.createSerializer(new ExecutionConfig()); + LinkedHashMap mockOriginalFieldToSerializerConfigSnapshot = new LinkedHashMap<>(mockOriginalFieldOrder.length); - for (Field field : mockOriginalFieldOrder) { - // just use forward compatible marker; just the field ordering is relevant for this test - mockOriginalFieldToSerializerConfigSnapshot.put(field, ForwardCompatibleSerializationFormatConfig.INSTANCE); - } + mockOriginalFieldToSerializerConfigSnapshot.put(mockOriginalFieldOrder[0], ser.getFieldSerializers()[3].snapshotConfiguration()); + mockOriginalFieldToSerializerConfigSnapshot.put(mockOriginalFieldOrder[1], ser.getFieldSerializers()[2].snapshotConfiguration()); + mockOriginalFieldToSerializerConfigSnapshot.put(mockOriginalFieldOrder[2], ser.getFieldSerializers()[5].snapshotConfiguration()); + mockOriginalFieldToSerializerConfigSnapshot.put(mockOriginalFieldOrder[3], ser.getFieldSerializers()[0].snapshotConfiguration()); + mockOriginalFieldToSerializerConfigSnapshot.put(mockOriginalFieldOrder[4], ser.getFieldSerializers()[1].snapshotConfiguration()); + mockOriginalFieldToSerializerConfigSnapshot.put(mockOriginalFieldOrder[5], ser.getFieldSerializers()[4].snapshotConfiguration()); PojoSerializer pojoSerializer = (PojoSerializer) type.createSerializer(new ExecutionConfig()); @@ -494,7 +498,9 @@ public void testReconfigureWithDifferentFieldOrder() throws Exception { new HashMap, TypeSerializerConfigSnapshot>()); // empty; irrelevant for this test // reconfigure - check reconfiguration result and that fields are reordered to the previous order - CompatibilityDecision strategy = pojoSerializer.getMigrationStrategyFor(mockPreviousConfigSnapshot); + CompatibilityDecision strategy = pojoSerializer.ensureCompatibility( + + mockPreviousConfigSnapshot); assertFalse(strategy.requireMigration()); int i = 0; for (Field field : mockOriginalFieldOrder) { diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerMigrationTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerMigrationTest.java index 83c82f60fafcf..fe44e0915dc0d 100644 --- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerMigrationTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/runtime/kryo/KryoSerializerMigrationTest.java @@ -109,7 +109,7 @@ public void testMigrationStrategyForDifferentRegistrationOrder() throws Exceptio } // reconfigure - check reconfiguration result and that registration id remains the same - CompatibilityDecision strategy = kryoSerializer.getMigrationStrategyFor(kryoSerializerConfigSnapshot); + CompatibilityDecision strategy = kryoSerializer.ensureCompatibility(kryoSerializerConfigSnapshot); assertFalse(strategy.requireMigration()); assertEquals(testClassId, kryoSerializer.getKryo().getRegistration(TestClass.class).getId()); assertEquals(testClassAId, kryoSerializer.getKryo().getRegistration(TestClassA.class).getId()); 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 9dd093d962f84..1917021b749b3 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,7 +395,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { throw new UnsupportedOperationException(); } } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java index c548d214dd7e0..a5438eaee2a10 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/NonDuplicatingTypeSerializer.java @@ -203,7 +203,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { throw new UnsupportedOperationException("This serializer is not registered for managed state."); } } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java index cc719f1bfb04d..f9ce6ae09167f 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/AbstractKeyedCEPPatternOperator.java @@ -507,9 +507,9 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof CollectionSerializerConfigSnapshot) { - CompatibilityDecision strategy = elementSerializer.getMigrationStrategyFor( + CompatibilityDecision strategy = elementSerializer.ensureCompatibility( ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); if (strategy.requireMigration()) { 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 1e849534efa12..a80a1235d848a 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 @@ -81,12 +81,12 @@ class CRowSerializer(val rowSerializer: TypeSerializer[Row]) extends TypeSeriali rowSerializer.snapshotConfiguration()) } - override protected def ensureCompatibility( + override def ensureCompatibility( configSnapshot: TypeSerializerConfigSnapshot): CompatibilityDecision[CRow] = { configSnapshot match { case crowSerializerConfigSnapshot: CRowSerializer.CRowSerializerConfigSnapshot => - val strategy = rowSerializer.getMigrationStrategyFor( + val strategy = rowSerializer.ensureCompatibility( crowSerializerConfigSnapshot.getSingleNestedSerializerConfigSnapshot) if (strategy.requireMigration()) { diff --git a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java index 866a1e3fc2fb3..c68a16ec43b44 100644 --- a/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java +++ b/flink-runtime/src/main/java/org/apache/flink/migration/MigrationNamespaceSerializerProxy.java @@ -110,7 +110,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { return CompatibilityDecision.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 29734c357c4c3..503a8e39b8451 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 @@ -146,9 +146,9 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof CollectionSerializerConfigSnapshot) { - CompatibilityDecision strategy = elementSerializer.getMigrationStrategyFor( + CompatibilityDecision strategy = elementSerializer.ensureCompatibility( ((CollectionSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); if (strategy.requireMigration()) { diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/state/HashMapSerializer.java b/flink-runtime/src/main/java/org/apache/flink/runtime/state/HashMapSerializer.java index 9767de405fa92..64ab570c1482b 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/state/HashMapSerializer.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/state/HashMapSerializer.java @@ -208,13 +208,13 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof MapSerializerConfigSnapshot) { TypeSerializerConfigSnapshot[] keyValueSerializerConfigSnapshots = ((MapSerializerConfigSnapshot) configSnapshot).getNestedSerializerConfigSnapshots(); - CompatibilityDecision keyStrategy = keySerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[0]); - CompatibilityDecision valueStrategy = valueSerializer.getMigrationStrategyFor(keyValueSerializerConfigSnapshots[1]); + CompatibilityDecision keyStrategy = keySerializer.ensureCompatibility(keyValueSerializerConfigSnapshots[0]); + CompatibilityDecision valueStrategy = valueSerializer.ensureCompatibility(keyValueSerializerConfigSnapshots[1]); if (keyStrategy.requireMigration() || valueStrategy.requireMigration()) { if (keyStrategy.getConvertDeserializer() != null && valueStrategy.getConvertDeserializer() != null) { 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 6ab228191c7fa..48abb6c2c8721 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 @@ -136,7 +136,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision 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 e523eb6ba6265..672ac1bb31532 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 @@ -145,7 +145,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision 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 06909f78ec45c..d4d730ec76ba4 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 @@ -113,7 +113,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { throw new UnsupportedOperationException(); } } 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 a381c2b8b374b..db1c43354127e 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 @@ -658,7 +658,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision 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 3c0db721bb0bf..5029997e4a630 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 @@ -150,7 +150,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { throw new UnsupportedOperationException(); } } 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 146ce0e67bbb5..931808bd13b77 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 @@ -116,7 +116,7 @@ class EitherSerializer[A, B, T <: Either[A, B]]( rightSerializer.snapshotConfiguration()) } - override protected def ensureCompatibility( + override def ensureCompatibility( configSnapshot: TypeSerializerConfigSnapshot): CompatibilityDecision[T] = { configSnapshot match { @@ -124,8 +124,8 @@ class EitherSerializer[A, B, T <: Either[A, B]]( val leftRightConfigs = eitherSerializerConfig.getNestedSerializerConfigSnapshots - val leftStrategy = leftSerializer.getMigrationStrategyFor(leftRightConfigs(0)) - val rightStrategy = rightSerializer.getMigrationStrategyFor(leftRightConfigs(1)) + val leftStrategy = leftSerializer.ensureCompatibility(leftRightConfigs(0)) + val rightStrategy = rightSerializer.ensureCompatibility(leftRightConfigs(1)) if (leftStrategy.requireMigration || rightStrategy.requireMigration) { if (leftStrategy.getConvertDeserializer != null 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 4af9b6d1ecd6d..0ddede7e81e8c 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 @@ -81,7 +81,7 @@ class EnumValueSerializer[E <: Enumeration](val enum: E) extends TypeSerializer[ enum.getClass.asInstanceOf[Class[E]]) } - override protected def ensureCompatibility( + override def ensureCompatibility( configSnapshot: TypeSerializerConfigSnapshot): CompatibilityDecision[E#Value] = { configSnapshot match { 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 0a37ea8c99a18..12fa6f6babcad 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 @@ -59,7 +59,7 @@ class NothingSerializer extends TypeSerializer[Any] { override def snapshotConfiguration(): TypeSerializerConfigSnapshot = throw new RuntimeException("This must not be used. You encountered a bug.") - override protected def ensureCompatibility( + override def ensureCompatibility( configSnapshot: TypeSerializerConfigSnapshot): CompatibilityDecision[Any] = throw new RuntimeException("This must not be used. You encountered a bug.") 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 10469a93b980f..cd44b58afbb2d 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 @@ -104,11 +104,11 @@ class OptionSerializer[A](val elemSerializer: TypeSerializer[A]) new OptionSerializer.OptionSerializerConfigSnapshot(elemSerializer.snapshotConfiguration()) } - override protected def ensureCompatibility( + override def ensureCompatibility( configSnapshot: TypeSerializerConfigSnapshot): CompatibilityDecision[Option[A]] = { configSnapshot match { case optionSerializerConfigSnapshot: OptionSerializer.OptionSerializerConfigSnapshot => - val strategy = elemSerializer.getMigrationStrategyFor( + val strategy = elemSerializer.ensureCompatibility( optionSerializerConfigSnapshot.getSingleNestedSerializerConfigSnapshot) if (strategy.requireMigration()) { 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 e51ba112834bd..8d5f567fee0ce 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 @@ -155,7 +155,7 @@ abstract class TraversableSerializer[T <: TraversableOnce[E], E]( throw new UnsupportedOperationException() } - override protected def ensureCompatibility( + override def ensureCompatibility( configSnapshot: TypeSerializerConfigSnapshot): CompatibilityDecision[T] = { throw new UnsupportedOperationException() } 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 0f0d4f16bba3c..683a1da92458c 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 @@ -110,7 +110,7 @@ class TrySerializer[A]( throwableSerializer.snapshotConfiguration()) } - override protected def ensureCompatibility( + override def ensureCompatibility( configSnapshot: TypeSerializerConfigSnapshot): CompatibilityDecision[Try[A]] = { configSnapshot match { @@ -119,9 +119,9 @@ class TrySerializer[A]( trySerializerConfigSnapshot.getNestedSerializerConfigSnapshots val elemStrategy = - elemSerializer.getMigrationStrategyFor(serializerConfigSnapshots(0)) + elemSerializer.ensureCompatibility(serializerConfigSnapshots(0)) val throwableStrategy = - throwableSerializer.getMigrationStrategyFor(serializerConfigSnapshots(1)) + throwableSerializer.ensureCompatibility(serializerConfigSnapshots(1)) if (elemStrategy.requireMigration() || throwableStrategy.requireMigration()) { CompatibilityDecision.requiresMigration(null) diff --git a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java index 8de22a9ebed3f..91aa726417975 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/MultiplexingStreamRecordSerializer.java @@ -219,9 +219,9 @@ public MultiplexingStreamRecordSerializerConfigSnapshot snapshotConfiguration() } @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof MultiplexingStreamRecordSerializerConfigSnapshot) { - CompatibilityDecision strategy = typeSerializer.getMigrationStrategyFor( + CompatibilityDecision strategy = typeSerializer.ensureCompatibility( ((MultiplexingStreamRecordSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); if (strategy.requireMigration()) { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java index 7a782c6c4d266..50e2fe65f9c15 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/migration/streaming/runtime/streamrecord/StreamRecordSerializer.java @@ -158,9 +158,9 @@ public StreamRecordSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof StreamRecordSerializerConfigSnapshot) { - CompatibilityDecision strategy = typeSerializer.getMigrationStrategyFor( + CompatibilityDecision strategy = typeSerializer.ensureCompatibility( ((StreamRecordSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); if (strategy.requireMigration()) { 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 80822408a72a2..ddc26926a040e 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 @@ -556,7 +556,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision> 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/api/operators/InternalTimer.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimer.java index 9d09753e99e4d..b9dc166af1acd 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimer.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/InternalTimer.java @@ -193,7 +193,7 @@ public TypeSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision> ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision> 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 4005e4b263471..e3453845bb818 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 @@ -280,9 +280,9 @@ public StreamElementSerializerConfigSnapshot snapshotConfiguration() { } @Override - protected CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { + public CompatibilityDecision ensureCompatibility(TypeSerializerConfigSnapshot configSnapshot) { if (configSnapshot instanceof StreamElementSerializerConfigSnapshot) { - CompatibilityDecision strategy = typeSerializer.getMigrationStrategyFor( + CompatibilityDecision strategy = typeSerializer.ensureCompatibility( ((StreamElementSerializerConfigSnapshot) configSnapshot).getSingleNestedSerializerConfigSnapshot()); if (strategy.requireMigration()) {