T
- The data type that the originating serializer of this configuration serializes.@PublicEvolving public interface TypeSerializerSnapshot<T>
TypeSerializerSnapshot
is a point-in-time view of a TypeSerializer
's configuration.
The configuration snapshot of a serializer is persisted within checkpoints
as a single source of meta information about the schema of serialized data in the checkpoint.
This serves three purposes:
Since serializer configuration snapshots needs to be used to ensure serialization compatibility for the same managed state as well as serving as a factory for compatible read serializers, the configuration snapshot should encode sufficient information about:
NOTE: Implementations must contain the default empty nullary constructor. This is required to be able to deserialize the configuration snapshot from its binary form.
Modifier and Type | Method and Description |
---|---|
int |
getCurrentVersion()
Returns the version of the current snapshot's written binary format.
|
void |
readSnapshot(int readVersion,
DataInputView in,
ClassLoader userCodeClassLoader)
Reads the serializer snapshot from the provided
DataInputView . |
static <T> TypeSerializerSnapshot<T> |
readVersionedSnapshot(DataInputView in,
ClassLoader cl)
Reads a snapshot from the stream, performing resolving
|
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.
|
void |
writeSnapshot(DataOutputView out)
Writes the serializer snapshot to the provided
DataOutputView . |
static void |
writeVersionedSnapshot(DataOutputView out,
TypeSerializerSnapshot<?> snapshot)
Writes the given snapshot to the out stream.
|
int getCurrentVersion()
void writeSnapshot(DataOutputView out) throws IOException
DataOutputView
.
The current version of the written serializer snapshot's binary format
is specified by the getCurrentVersion()
method.out
- the DataOutputView
to write the snapshot to.IOException
- Thrown if the snapshot data could not be written.void readSnapshot(int readVersion, DataInputView in, ClassLoader userCodeClassLoader) throws IOException
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.readVersion
- version of the serializer snapshot's written binary formatin
- the DataInputView
to read the snapshot from.userCodeClassLoader
- the user code classloader
* @throws IOException Thrown if the snapshot data could be read or parsed.IOException
TypeSerializer<T> restoreSerializer()
TypeSerializerSchemaCompatibility<T> resolveSchemaCompatibility(TypeSerializer<T> newSerializer)
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.
newSerializer
- the new serializer to check.static void writeVersionedSnapshot(DataOutputView out, TypeSerializerSnapshot<?> snapshot) throws IOException
writeSnapshot(DataOutputView)
.
The snapshot written with this method can be read via readVersionedSnapshot(DataInputView, ClassLoader)
.
IOException
static <T> TypeSerializerSnapshot<T> readVersionedSnapshot(DataInputView in, ClassLoader cl) throws IOException
This method reads snapshots written by writeVersionedSnapshot(DataOutputView, TypeSerializerSnapshot)
.
IOException
Copyright © 2014–2020 The Apache Software Foundation. All rights reserved.