T
- The data type that the originating serializer of this snapshot serializes.S
- The type of the originating serializer.@PublicEvolving public abstract class CompositeTypeSerializerSnapshot<T,S extends TypeSerializer<T>> extends Object implements TypeSerializerSnapshot<T>
CompositeTypeSerializerSnapshot
is a convenient serializer snapshot class that can be
used by simple serializers which 1) delegates its serialization to multiple nested serializers,
and 2) may contain some extra static information that needs to be persisted as part of its
snapshot.
Examples for this would be the ListSerializer
, MapSerializer
, EitherSerializer
, etc., in which case the serializer, called the "outer" serializer in this
context, has only some nested serializers that needs to be persisted as its snapshot, and nothing
else that needs to be persisted as the "outer" snapshot. An example which has non-empty outer
snapshots would be the GenericArraySerializer
, which beyond the nested component
serializer, also contains a class of the component type that needs to be persisted.
Serializers that do have some outer snapshot needs to make sure to implement the methods
writeOuterSnapshot(DataOutputView)
, readOuterSnapshot(int, DataInputView,
ClassLoader)
, and resolveOuterSchemaCompatibility(TypeSerializer)
(TypeSerializer)}
when using this class as the base for its serializer snapshot class. By default, the base
implementations of these methods are empty, i.e. this class assumes that subclasses do not have
any outer snapshot that needs to be persisted.
This base class has its own versioning for the format in which it writes the outer snapshot
and the nested serializer snapshots. The version of the serialization format of this based class
is defined by getCurrentVersion()
. This is independent of the version in which
subclasses writes their outer snapshot, defined by getCurrentOuterSnapshotVersion()
.
This means that the outer snapshot's version can be maintained only taking into account changes
in how the outer snapshot is written. Any changes in the base format does not require upticks in
the outer snapshot's version.
The current version of the serialization format of a CompositeTypeSerializerSnapshot
is as follows:
+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
| CompositeTypeSerializerSnapshot | CompositeTypeSerializerSnapshot | Outer snapshot |
| version | MAGIC_NUMBER | version |
+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
| Outer snapshot |
| #writeOuterSnapshot(DataOutputView out) |
+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
| Delegate MAGIC_NUMBER | Delegate version | Num. nested serializers |
+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
| Nested serializer snapshots |
+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+-+
Modifier and Type | Class and Description |
---|---|
protected static class |
CompositeTypeSerializerSnapshot.OuterSchemaCompatibility
Indicates schema compatibility of the serializer configuration persisted as the outer
snapshot.
|
Constructor and Description |
---|
CompositeTypeSerializerSnapshot(Class<? extends TypeSerializer> correspondingSerializerClass)
Constructor to be used for read instantiation.
|
CompositeTypeSerializerSnapshot(S serializerInstance)
Constructor to be used for writing the snapshot.
|
Modifier and Type | Method and Description |
---|---|
protected abstract S |
createOuterSerializerWithNestedSerializers(TypeSerializer<?>[] nestedSerializers)
Creates an instance of the outer serializer with a given array of its nested serializers.
|
protected abstract int |
getCurrentOuterSnapshotVersion()
Returns the version of the current outer snapshot's written binary format.
|
int |
getCurrentVersion()
Returns the version of the current snapshot's written binary format.
|
protected abstract TypeSerializer<?>[] |
getNestedSerializers(S outerSerializer)
Gets the nested serializers from the outer serializer.
|
TypeSerializerSnapshot<?>[] |
getNestedSerializerSnapshots() |
protected boolean |
isOuterSnapshotCompatible(S newSerializer)
Deprecated.
this method is deprecated, and will be removed in the future. Please implement
resolveOuterSchemaCompatibility(TypeSerializer) instead. |
protected void |
readOuterSnapshot(int readOuterSnapshotVersion,
DataInputView in,
ClassLoader userCodeClassLoader)
Reads the outer snapshot, i.e.
|
void |
readSnapshot(int readVersion,
DataInputView in,
ClassLoader userCodeClassLoader)
Reads the serializer snapshot from the provided
DataInputView . |
protected CompositeTypeSerializerSnapshot.OuterSchemaCompatibility |
resolveOuterSchemaCompatibility(S newSerializer)
Checks the schema compatibility of the given new serializer based on the outer snapshot.
|
TypeSerializerSchemaCompatibility<T> |
resolveSchemaCompatibility(TypeSerializer<T> newSerializer)
Checks a new serializer's compatibility to read data written by the prior serializer.
|
TypeSerializer<T> |
restoreSerializer()
Recreates a serializer instance from this snapshot.
|
protected void |
writeOuterSnapshot(DataOutputView out)
Writes the outer snapshot, i.e.
|
void |
writeSnapshot(DataOutputView out)
Writes the serializer snapshot to the provided
DataOutputView . |
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
readVersionedSnapshot, writeVersionedSnapshot
public CompositeTypeSerializerSnapshot(Class<? extends TypeSerializer> correspondingSerializerClass)
correspondingSerializerClass
- the expected class of the new serializer.public CompositeTypeSerializerSnapshot(S serializerInstance)
serializerInstance
- an instance of the originating serializer of this snapshot.public final int getCurrentVersion()
TypeSerializerSnapshot
getCurrentVersion
in interface TypeSerializerSnapshot<T>
public final void writeSnapshot(DataOutputView out) throws IOException
TypeSerializerSnapshot
DataOutputView
. The current version of
the written serializer snapshot's binary format is specified by the TypeSerializerSnapshot.getCurrentVersion()
method.writeSnapshot
in interface TypeSerializerSnapshot<T>
out
- the DataOutputView
to write the snapshot to.IOException
- Thrown if the snapshot data could not be written.TypeSerializerSnapshot.writeVersionedSnapshot(DataOutputView, TypeSerializerSnapshot)
public final void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException
TypeSerializerSnapshot
DataInputView
. The version of the
binary format that the serializer snapshot was written with is provided. This version can be
used to determine how the serializer snapshot should be read.readSnapshot
in interface TypeSerializerSnapshot<T>
readVersion
- version of the serializer snapshot's written binary formatin
- the DataInputView
to read the snapshot from.userCodeClassLoader
- the user code classloaderIOException
- Thrown if the snapshot data could be read or parsed.TypeSerializerSnapshot.readVersionedSnapshot(DataInputView, ClassLoader)
public TypeSerializerSnapshot<?>[] getNestedSerializerSnapshots()
public final TypeSerializerSchemaCompatibility<T> resolveSchemaCompatibility(TypeSerializer<T> newSerializer)
TypeSerializerSnapshot
When a checkpoint/savepoint is restored, this method checks whether the serialization format of the data in the checkpoint/savepoint is compatible for the format of the serializer used by the program that restores the checkpoint/savepoint. The outcome can be that the serialization format is compatible, that the program's serializer needs to reconfigure itself (meaning to incorporate some information from the TypeSerializerSnapshot to be compatible), that the format is outright incompatible, or that a migration needed. In the latter case, the TypeSerializerSnapshot produces a serializer to deserialize the data, and the restoring program's serializer re-serializes the data, thus converting the format during the restore operation.
resolveSchemaCompatibility
in interface TypeSerializerSnapshot<T>
newSerializer
- the new serializer to check.public final TypeSerializer<T> restoreSerializer()
TypeSerializerSnapshot
restoreSerializer
in interface TypeSerializerSnapshot<T>
protected abstract int getCurrentOuterSnapshotVersion()
protected abstract TypeSerializer<?>[] getNestedSerializers(S outerSerializer)
outerSerializer
- the outer serializer.protected abstract S createOuterSerializerWithNestedSerializers(TypeSerializer<?>[] nestedSerializers)
nestedSerializers
- array of nested serializers to create the outer serializer with.protected void writeOuterSnapshot(DataOutputView out) throws IOException
The base implementation of this methods writes nothing, i.e. it assumes that the outer
serializer only has nested serializers and no extra information. Otherwise, if the outer
serializer contains some extra information that needs to be persisted as part of the
serializer snapshot, this must be overridden. Note that this method and the corresponding
methods readOuterSnapshot(int, DataInputView, ClassLoader)
, resolveOuterSchemaCompatibility(TypeSerializer)
needs to be implemented.
out
- the DataOutputView
to write the outer snapshot to.IOException
protected void readOuterSnapshot(int readOuterSnapshotVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException
The base implementation of this methods reads nothing, i.e. it assumes that the outer
serializer only has nested serializers and no extra information. Otherwise, if the outer
serializer contains some extra information that has been persisted as part of the serializer
snapshot, this must be overridden. Note that this method and the corresponding methods writeOuterSnapshot(DataOutputView)
, resolveOuterSchemaCompatibility(TypeSerializer)
needs to be implemented.
readOuterSnapshotVersion
- the read version of the outer snapshot.in
- the DataInputView
to read the outer snapshot from.userCodeClassLoader
- the user code class loader.IOException
@Deprecated protected boolean isOuterSnapshotCompatible(S newSerializer)
resolveOuterSchemaCompatibility(TypeSerializer)
instead.The base implementation of this method just returns true
, i.e. it assumes that the
outer serializer only has nested serializers and no extra information, and therefore the
result of the check must always be true. Otherwise, if the outer serializer contains some
extra information that has been persisted as part of the serializer snapshot, this must be
overridden. Note that this method and the corresponding methods writeOuterSnapshot(DataOutputView)
, readOuterSnapshot(int, DataInputView,
ClassLoader)
needs to be implemented.
newSerializer
- the new serializer, which contains the new outer information to check
against.protected CompositeTypeSerializerSnapshot.OuterSchemaCompatibility resolveOuterSchemaCompatibility(S newSerializer)
The base implementation of this method assumes that the outer serializer only has nested
serializers and no extra information, and therefore the result of the check is CompositeTypeSerializerSnapshot.OuterSchemaCompatibility.COMPATIBLE_AS_IS
. Otherwise, if the outer serializer contains some
extra information that has been persisted as part of the serializer snapshot, this must be
overridden. Note that this method and the corresponding methods writeOuterSnapshot(DataOutputView)
, readOuterSnapshot(int, DataInputView,
ClassLoader)
needs to be implemented.
newSerializer
- the new serializer, which contains the new outer information to check
against.CompositeTypeSerializerSnapshot.OuterSchemaCompatibility
indicating whether or the new serializer's outer
information is compatible, requires migration, or incompatible with the one written in
this snapshot.Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.