@Internal public class BinaryRowDataSerializer extends AbstractRowDataSerializer<BinaryRowData>
BinaryRowData
.Modifier and Type | Class and Description |
---|---|
static class |
BinaryRowDataSerializer.BinaryRowDataSerializerSnapshot
|
Modifier and Type | Field and Description |
---|---|
static int |
LENGTH_SIZE_IN_BYTES |
Constructor and Description |
---|
BinaryRowDataSerializer(int numFields) |
Modifier and Type | Method and Description |
---|---|
void |
checkSkipReadForFixLengthPart(AbstractPagedInputView source)
We need skip bytes to read when the remain bytes of current segment is not enough to write
binary row fixed part.
|
BinaryRowData |
copy(BinaryRowData from)
Creates a deep copy of the given element in a new element.
|
BinaryRowData |
copy(BinaryRowData from,
BinaryRowData reuse)
Creates a copy from the given element.
|
void |
copy(DataInputView source,
DataOutputView target)
Copies exactly one record from the source input view to the target output view.
|
void |
copyFromPagesToView(AbstractPagedInputView source,
DataOutputView target)
Copy a binaryRow which stored in paged input view to output view.
|
BinaryRowData |
createInstance()
Creates a new instance of the data type.
|
BinaryRowData |
deserialize(BinaryRowData reuse,
DataInputView source)
De-serializes a record from the given source input view into the given reuse record instance
if mutable.
|
BinaryRowData |
deserialize(DataInputView source)
De-serializes a record from the given source input view.
|
BinaryRowData |
deserializeFromPages(AbstractPagedInputView headerLessView)
De-serializes a record from the given source paged input view.
|
BinaryRowData |
deserializeFromPages(BinaryRowData reuse,
AbstractPagedInputView headerLessView)
Reuse version of
PagedTypeSerializer.deserializeFromPages(AbstractPagedInputView) . |
TypeSerializer<BinaryRowData> |
duplicate()
Creates a deep copy of this serializer if it is necessary, i.e.
|
boolean |
equals(Object obj) |
int |
getArity()
Get the number of fields.
|
int |
getFixedLengthPartSize() |
int |
getLength()
Gets the length of the data type, if it is a fix length data type.
|
int |
getSerializedRowFixedPartLength()
Return fixed part length to serialize one row.
|
int |
hashCode() |
boolean |
isImmutableType()
Gets whether the type is an immutable type.
|
BinaryRowData |
mapFromPages(BinaryRowData reuse,
AbstractPagedInputView headerLessView)
Map a reused record from the given source paged input view.
|
void |
pointTo(int length,
BinaryRowData reuse,
AbstractPagedInputView headerLessView)
Point row to memory segments with offset(in the AbstractPagedInputView) and length.
|
void |
serialize(BinaryRowData record,
DataOutputView target)
Serializes the given record to the given target output view.
|
int |
serializeToPages(BinaryRowData record,
AbstractPagedOutputView headerLessView)
Serializes the given record to the given target paged output view.
|
static void |
serializeWithoutLengthSlow(BinaryRowData record,
MemorySegmentWritable out) |
void |
skipRecordFromPages(AbstractPagedInputView headerLessView)
Skip over bytes of one record from the paged input view, discarding the skipped bytes.
|
TypeSerializerSnapshot<BinaryRowData> |
snapshotConfiguration()
Snapshots the configuration of this TypeSerializer.
|
BinaryRowData |
toBinaryRow(BinaryRowData rowData)
Convert a
RowData to a BinaryRowData . |
public static final int LENGTH_SIZE_IN_BYTES
public boolean isImmutableType()
TypeSerializer
isImmutableType
in class TypeSerializer<BinaryRowData>
public TypeSerializer<BinaryRowData> duplicate()
TypeSerializer
We need this because Serializers might be used in several threads. Stateless serializers are inherently thread-safe while stateful serializers might not be thread-safe.
duplicate
in class TypeSerializer<BinaryRowData>
public BinaryRowData createInstance()
TypeSerializer
createInstance
in class TypeSerializer<BinaryRowData>
public BinaryRowData copy(BinaryRowData from)
TypeSerializer
copy
in class TypeSerializer<BinaryRowData>
from
- The element reuse be copied.public BinaryRowData copy(BinaryRowData from, BinaryRowData reuse)
TypeSerializer
copy
in class TypeSerializer<BinaryRowData>
from
- The element to be copied.reuse
- The element to be reused. May or may not be used.public int getLength()
TypeSerializer
getLength
in class TypeSerializer<BinaryRowData>
-1
for variable length data types.public void serialize(BinaryRowData record, DataOutputView target) throws IOException
TypeSerializer
serialize
in class TypeSerializer<BinaryRowData>
record
- The record to serialize.target
- The output view to write the serialized data to.IOException
- Thrown, if the serialization encountered an I/O related error. Typically
raised by the output view, which may have an underlying I/O channel to which it
delegates.public BinaryRowData deserialize(DataInputView source) throws IOException
TypeSerializer
deserialize
in class TypeSerializer<BinaryRowData>
source
- The input view from which to read the data.IOException
- Thrown, if the de-serialization encountered an I/O related error.
Typically raised by the input view, which may have an underlying I/O channel from which
it reads.public BinaryRowData deserialize(BinaryRowData reuse, DataInputView source) throws IOException
TypeSerializer
deserialize
in class TypeSerializer<BinaryRowData>
reuse
- The record instance into which to de-serialize the data.source
- The input view from which to read the data.IOException
- Thrown, if the de-serialization encountered an I/O related error.
Typically raised by the input view, which may have an underlying I/O channel from which
it reads.public int getArity()
AbstractRowDataSerializer
getArity
in class AbstractRowDataSerializer<BinaryRowData>
public BinaryRowData toBinaryRow(BinaryRowData rowData) throws IOException
AbstractRowDataSerializer
RowData
to a BinaryRowData
.toBinaryRow
in class AbstractRowDataSerializer<BinaryRowData>
IOException
public int serializeToPages(BinaryRowData record, AbstractPagedOutputView headerLessView) throws IOException
PagedTypeSerializer
BinaryRowData
.serializeToPages
in class PagedTypeSerializer<BinaryRowData>
record
- The record to serialize.headerLessView
- The output view to write the serialized data to.IOException
- Thrown, if the serialization encountered an I/O related error. Typically
raised by the output view, which may have an underlying I/O channel to which it
delegates.public static void serializeWithoutLengthSlow(BinaryRowData record, MemorySegmentWritable out) throws IOException
IOException
public BinaryRowData deserializeFromPages(AbstractPagedInputView headerLessView) throws IOException
PagedTypeSerializer
BinaryRowData
. Typically, the content read from source should be copied out when
de-serializing, and we are not expecting the underlying data from source is reused. If you
have such requirement, see #mapFromPages(T, AbstractPagedInputView)
.deserializeFromPages
in class PagedTypeSerializer<BinaryRowData>
headerLessView
- The input view from which to read the data.IOException
- Thrown, if the de-serialization encountered an I/O related error.
Typically raised by the input view, which may have an underlying I/O channel from which
it reads.public BinaryRowData deserializeFromPages(BinaryRowData reuse, AbstractPagedInputView headerLessView) throws IOException
PagedTypeSerializer
PagedTypeSerializer.deserializeFromPages(AbstractPagedInputView)
.deserializeFromPages
in class PagedTypeSerializer<BinaryRowData>
IOException
public BinaryRowData mapFromPages(BinaryRowData reuse, AbstractPagedInputView headerLessView) throws IOException
PagedTypeSerializer
If you choose the zero copy way, you have to deal with the lifecycle of the pages properly.
mapFromPages
in class PagedTypeSerializer<BinaryRowData>
reuse
- the reused record to be mappedheaderLessView
- The input view from which to read the data.IOException
- Thrown, if the de-serialization encountered an I/O related error.
Typically raised by the input view, which may have an underlying I/O channel from which
it reads.public void skipRecordFromPages(AbstractPagedInputView headerLessView) throws IOException
PagedTypeSerializer
skipRecordFromPages
in class PagedTypeSerializer<BinaryRowData>
IOException
public void copyFromPagesToView(AbstractPagedInputView source, DataOutputView target) throws IOException
source
- source paged input view where the binary row storedtarget
- the target output view.IOException
public void pointTo(int length, BinaryRowData reuse, AbstractPagedInputView headerLessView) throws IOException
length
- row length.reuse
- reuse BinaryRowData object.headerLessView
- source memory segments container.IOException
public void checkSkipReadForFixLengthPart(AbstractPagedInputView source) throws IOException
BinaryRowData
.IOException
public int getSerializedRowFixedPartLength()
public int getFixedLengthPartSize()
public void copy(DataInputView source, DataOutputView target) throws IOException
TypeSerializer
target.write(source, 8);
.copy
in class TypeSerializer<BinaryRowData>
source
- The input view from which to read the record.target
- The target output view to which to write the record.IOException
- Thrown if any of the two views raises an exception.public boolean equals(Object obj)
equals
in class TypeSerializer<BinaryRowData>
public int hashCode()
hashCode
in class TypeSerializer<BinaryRowData>
public TypeSerializerSnapshot<BinaryRowData> snapshotConfiguration()
TypeSerializer
The snapshot of the TypeSerializer is supposed to contain all information that affects the serialization format of the serializer. The snapshot serves two purposes: First, to reproduce the serializer when the checkpoint/savepoint is restored, and second, to check whether the serialization format is compatible with the serializer used in the restored program.
IMPORTANT: TypeSerializerSnapshots changed after Flink 1.6. Serializers implemented against Flink versions up to 1.6 should still work, but adjust to new model to enable state evolution and be future-proof. See the class-level comments, section "Upgrading TypeSerializers to the new TypeSerializerSnapshot model" for details.
snapshotConfiguration
in class TypeSerializer<BinaryRowData>
null
).TypeSerializerSnapshot.resolveSchemaCompatibility(TypeSerializer)
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.