Skip to content

Commit

Permalink
[FLINK-1005] Extend TypeSerializer interface to handle non-mutable ob…
Browse files Browse the repository at this point in the history
…ject deserialization more efficiently.
  • Loading branch information
StephanEwen committed Sep 30, 2014
1 parent ea4c882 commit 76d4a75
Show file tree
Hide file tree
Showing 61 changed files with 1,405 additions and 301 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,15 @@ public StreamRecord<T> createInstance() {
throw new RuntimeException("Cannot instantiate StreamRecord.", e);
}
}

@Override
public StreamRecord<T> copy(StreamRecord<T> from) {
StreamRecord<T> rec = new StreamRecord<T>();
rec.isTuple = from.isTuple;
rec.setId(from.getId().copy());
rec.setObject(typeSerializer.copy(from.getObject()));
return rec;
}

@Override
public StreamRecord<T> copy(StreamRecord<T> from, StreamRecord<T> reuse) {
Expand All @@ -81,10 +90,18 @@ public void serialize(StreamRecord<T> value, DataOutputView target) throws IOExc
value.getId().write(target);
typeSerializer.serialize(value.getObject(), target);
}

@Override
public StreamRecord<T> deserialize(DataInputView source) throws IOException {
StreamRecord<T> record = new StreamRecord<T>();
record.isTuple = this.isTuple;
record.getId().read(source);
record.setObject(typeSerializer.deserialize(source));
return record;
}

@Override
public StreamRecord<T> deserialize(StreamRecord<T> reuse, DataInputView source)
throws IOException {
public StreamRecord<T> deserialize(StreamRecord<T> reuse, DataInputView source) throws IOException {
reuse.getId().read(source);
reuse.setObject(typeSerializer.deserialize(reuse.getObject(), source));
return reuse;
Expand All @@ -94,5 +111,4 @@ public StreamRecord<T> deserialize(StreamRecord<T> reuse, DataInputView source)
public void copy(DataInputView source, DataOutputView target) throws IOException {
// Needs to be implemented
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -73,10 +73,21 @@ public abstract class TypeSerializer<T> implements Serializable {
public abstract T createInstance();

/**
* Creates a copy from the given element, storing the copied result in the given reuse element if type is mutable.
* Creates a deep copy of the given element in a new element.
*
* @param from The element reuse be copied.
* @param reuse The element to be reused.
* @return A deep copy of the element.
*/
public abstract T copy(T from);

/**
* Creates a copy from the given element.
* The method makes an attempt to store the copy in the given reuse element, if the type is mutable.
* This is, however, not guaranteed.
*
* @param from The element to be copied.
* @param reuse The element to be reused. May or may not be used.
* @return A deep copy of the element.
*/
public abstract T copy(T from, T reuse);

Expand All @@ -102,11 +113,23 @@ public abstract class TypeSerializer<T> implements Serializable {
*/
public abstract void serialize(T record, DataOutputView target) throws IOException;

/**
* De-serializes a record from the given source input view.
*
* @param source The input view from which to read the data.
* @result The deserialized element.
*
* @throws 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 abstract T deserialize(DataInputView source) throws IOException;

/**
* De-serializes a record from the given source input view into the given reuse record instance if mutable.
*
* @param reuse The record instance into which to de-serialize the data.
* @param source The input view from which to read the data.
* @result The deserialized element.
*
* @throws 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.
Expand All @@ -126,19 +149,4 @@ public abstract class TypeSerializer<T> implements Serializable {
* @throws IOException Thrown if any of the two views raises an exception.
*/
public abstract void copy(DataInputView source, DataOutputView target) throws IOException;

// --------------------------------------------------------------------------------------------
// Default Utilities: Hash code and equals are pre-defined for singleton serializers, where
// all instances are equal
// --------------------------------------------------------------------------------------------

@Override
public int hashCode() {
return getClass().hashCode();
}

@Override
public boolean equals(Object obj) {
return obj != null && obj.getClass() == this.getClass();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,10 @@

import java.io.IOException;

import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;


public class BooleanSerializer extends TypeSerializer<Boolean> {
public final class BooleanSerializer extends TypeSerializerSingleton<Boolean> {

private static final long serialVersionUID = 1L;

Expand All @@ -48,6 +46,11 @@ public Boolean createInstance() {
return FALSE;
}

@Override
public Boolean copy(Boolean from) {
return from;
}

@Override
public Boolean copy(Boolean from, Boolean reuse) {
return from;
Expand All @@ -63,6 +66,11 @@ public void serialize(Boolean record, DataOutputView target) throws IOException
target.writeBoolean(record.booleanValue());
}

@Override
public Boolean deserialize(DataInputView source) throws IOException {
return Boolean.valueOf(source.readBoolean());
}

@Override
public Boolean deserialize(Boolean reuse, DataInputView source) throws IOException {
return Boolean.valueOf(source.readBoolean());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,12 @@

import java.io.IOException;

import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.types.BooleanValue;


public class BooleanValueSerializer extends TypeSerializer<BooleanValue> {
public final class BooleanValueSerializer extends TypeSerializerSingleton<BooleanValue> {

private static final long serialVersionUID = 1L;

Expand All @@ -48,6 +47,13 @@ public BooleanValue createInstance() {
return new BooleanValue();
}

@Override
public BooleanValue copy(BooleanValue from) {
BooleanValue result = new BooleanValue();
result.setValue(from.getValue());
return result;
}

@Override
public BooleanValue copy(BooleanValue from, BooleanValue reuse) {
reuse.setValue(from.getValue());
Expand All @@ -64,6 +70,11 @@ public void serialize(BooleanValue record, DataOutputView target) throws IOExcep
record.write(target);
}

@Override
public BooleanValue deserialize(DataInputView source) throws IOException {
return deserialize(new BooleanValue(), source);
}

@Override
public BooleanValue deserialize(BooleanValue reuse, DataInputView source) throws IOException {
reuse.read(source);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,11 @@

import java.io.IOException;

import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;


public class ByteSerializer extends TypeSerializer<Byte> {
public final class ByteSerializer extends TypeSerializerSingleton<Byte> {

private static final long serialVersionUID = 1L;

Expand All @@ -49,6 +48,11 @@ public Byte createInstance() {
return ZERO;
}

@Override
public Byte copy(Byte from) {
return from;
}

@Override
public Byte copy(Byte from, Byte reuse) {
return from;
Expand All @@ -65,9 +69,14 @@ public void serialize(Byte record, DataOutputView target) throws IOException {
}

@Override
public Byte deserialize(Byte reuse, DataInputView source) throws IOException {
public Byte deserialize(DataInputView source) throws IOException {
return Byte.valueOf(source.readByte());
}

@Override
public Byte deserialize(Byte reuse, DataInputView source) throws IOException {
return deserialize(source);
}

@Override
public void copy(DataInputView source, DataOutputView target) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,12 @@

import java.io.IOException;

import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.types.ByteValue;


public class ByteValueSerializer extends TypeSerializer<ByteValue> {
public final class ByteValueSerializer extends TypeSerializerSingleton<ByteValue> {

private static final long serialVersionUID = 1L;

Expand All @@ -48,6 +47,11 @@ public ByteValue createInstance() {
return new ByteValue();
}

@Override
public ByteValue copy(ByteValue from) {
return copy(from, new ByteValue());
}

@Override
public ByteValue copy(ByteValue from, ByteValue reuse) {
reuse.setValue(from.getValue());
Expand All @@ -64,6 +68,11 @@ public void serialize(ByteValue record, DataOutputView target) throws IOExceptio
record.write(target);
}

@Override
public ByteValue deserialize(DataInputView source) throws IOException {
return deserialize(new ByteValue(), source);
}

@Override
public ByteValue deserialize(ByteValue reuse, DataInputView source) throws IOException {
reuse.read(source);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,11 @@

import java.io.IOException;

import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;


public class CharSerializer extends TypeSerializer<Character> {
public final class CharSerializer extends TypeSerializerSingleton<Character> {

private static final long serialVersionUID = 1L;

Expand All @@ -49,6 +48,11 @@ public Character createInstance() {
return ZERO;
}

@Override
public Character copy(Character from) {
return from;
}

@Override
public Character copy(Character from, Character reuse) {
return from;
Expand All @@ -65,9 +69,14 @@ public void serialize(Character record, DataOutputView target) throws IOExceptio
}

@Override
public Character deserialize(Character reuse, DataInputView source) throws IOException {
public Character deserialize(DataInputView source) throws IOException {
return Character.valueOf(source.readChar());
}

@Override
public Character deserialize(Character reuse, DataInputView source) throws IOException {
return deserialize(source);
}

@Override
public void copy(DataInputView source, DataOutputView target) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,13 +20,11 @@

import java.io.IOException;

import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.types.CharValue;


public class CharValueSerializer extends TypeSerializer<CharValue> {
public class CharValueSerializer extends TypeSerializerSingleton<CharValue> {

private static final long serialVersionUID = 1L;

Expand All @@ -47,6 +45,11 @@ public boolean isStateful() {
public CharValue createInstance() {
return new CharValue();
}

@Override
public CharValue copy(CharValue from) {
return copy(from, new CharValue());
}

@Override
public CharValue copy(CharValue from, CharValue reuse) {
Expand All @@ -63,6 +66,11 @@ public int getLength() {
public void serialize(CharValue record, DataOutputView target) throws IOException {
record.write(target);
}

@Override
public CharValue deserialize(DataInputView source) throws IOException {
return deserialize(new CharValue(), source);
}

@Override
public CharValue deserialize(CharValue reuse, DataInputView source) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,12 +20,10 @@

import java.io.IOException;

import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.core.memory.DataInputView;
import org.apache.flink.core.memory.DataOutputView;


public class DoubleSerializer extends TypeSerializer<Double> {
public final class DoubleSerializer extends TypeSerializerSingleton<Double> {

private static final long serialVersionUID = 1L;

Expand All @@ -49,6 +47,11 @@ public Double createInstance() {
return ZERO;
}

@Override
public Double copy(Double from) {
return from;
}

@Override
public Double copy(Double from, Double reuse) {
return from;
Expand All @@ -65,9 +68,14 @@ public void serialize(Double record, DataOutputView target) throws IOException {
}

@Override
public Double deserialize(Double reuse, DataInputView source) throws IOException {
public Double deserialize(DataInputView source) throws IOException {
return Double.valueOf(source.readDouble());
}

@Override
public Double deserialize(Double reuse, DataInputView source) throws IOException {
return deserialize(source);
}

@Override
public void copy(DataInputView source, DataOutputView target) throws IOException {
Expand Down
Loading

0 comments on commit 76d4a75

Please sign in to comment.