Skip to content

Commit

Permalink
[FLINK-2283] [streaming] Proper serialization of state in StreamGroup…
Browse files Browse the repository at this point in the history
…edFold and Reduce
  • Loading branch information
mbalassi committed Oct 6, 2015
1 parent a76d963 commit c414ea9
Show file tree
Hide file tree
Showing 12 changed files with 215 additions and 107 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ public interface StateCheckpointer<S, C extends Serializable> {
*
* @return A snapshot of the operator state.
*/
public C snapshotState(S state, long checkpointId, long checkpointTimestamp);
C snapshotState(S state, long checkpointId, long checkpointTimestamp);

/**
* Restores the operator states from a given snapshot. The restores state
Expand All @@ -69,5 +69,5 @@ public interface StateCheckpointer<S, C extends Serializable> {
* The state snapshot that needs to be restored.
* @return The state corresponding to the snapshot.
*/
public S restoreState(C stateSnapshot);
S restoreState(C stateSnapshot);
}
Original file line number Diff line number Diff line change
Expand Up @@ -186,7 +186,8 @@ public <W extends Window> WindowedStream<T, KEY, W> window(WindowAssigner<? supe
* @return The transformed DataStream.
*/
public SingleOutputStreamOperator<T, ?> reduce(ReduceFunction<T> reducer) {
return transform("Keyed Reduce", getType(), new StreamGroupedReduce<>(clean(reducer), keySelector));
return transform("Keyed Reduce", getType(), new StreamGroupedReduce<T>(
clean(reducer), keySelector, getType()));
}

/**
Expand All @@ -208,7 +209,7 @@ public <W extends Window> WindowedStream<T, KEY, W> window(WindowAssigner<? supe
Utils.getCallLocationName(), true);

return transform("Keyed Fold", outType, new StreamGroupedFold<>(clean(folder),
keySelector, initialValue));
keySelector, initialValue, getType()));
}

/**
Expand Down Expand Up @@ -443,7 +444,7 @@ public <W extends Window> WindowedStream<T, KEY, W> window(WindowAssigner<? supe
}

protected SingleOutputStreamOperator<T, ?> aggregate(AggregationFunction<T> aggregate) {
StreamGroupedReduce<T> operator = new StreamGroupedReduce<>(clean(aggregate), keySelector);
StreamGroupedReduce<T> operator = new StreamGroupedReduce<T>(clean(aggregate), keySelector, getType());
return transform("Keyed Aggregation", getType(), operator);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,6 @@ public abstract class AbstractStreamOperator<OUT> implements StreamOperator<OUT>

private static final long serialVersionUID = 1L;


protected transient StreamingRuntimeContext runtimeContext;

protected transient ExecutionConfig executionConfig;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -56,8 +56,7 @@ public abstract class AbstractUdfStreamOperator<OUT, F extends Function>
private static final long serialVersionUID = 1L;

private static final Logger LOG = LoggerFactory.getLogger(AbstractUdfStreamOperator.class);



/** the user function */
protected final F userFunction;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,13 +30,15 @@
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.core.memory.InputViewDataInputStreamWrapper;
import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;
import org.apache.flink.streaming.api.state.KVMapCheckpointer;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;

public class StreamGroupedFold<IN, OUT> extends AbstractUdfStreamOperator<OUT, FoldFunction<IN, OUT>>

implements OneInputStreamOperator<IN, OUT>, OutputTypeConfigurable<OUT> {

private static final long serialVersionUID = 1L;
Expand All @@ -50,13 +52,18 @@ public class StreamGroupedFold<IN, OUT> extends AbstractUdfStreamOperator<OUT, F
private TypeSerializer<OUT> outTypeSerializer;
private transient OUT initialValue;

public StreamGroupedFold(
FoldFunction<IN, OUT> folder,
KeySelector<IN, ?> keySelector,
OUT initialValue) {
// Store the typeinfo, create serializer during runtime
private TypeInformation<Object> keyTypeInformation;

@SuppressWarnings("unchecked")
public StreamGroupedFold(FoldFunction<IN, OUT> folder, KeySelector<IN, ?> keySelector,
OUT initialValue, TypeInformation<IN> inTypeInformation) {
super(folder);
this.keySelector = keySelector;
this.initialValue = initialValue;
keyTypeInformation = (TypeInformation<Object>) TypeExtractor
.getKeySelectorTypes(keySelector, inTypeInformation);

}

@Override
Expand All @@ -75,7 +82,9 @@ public void open(Configuration configuration) throws Exception {
initialValue = outTypeSerializer.deserialize(in);

values = runtimeContext.getOperatorState("flink_internal_fold_values",
new HashMap<Object, OUT>(), false);
new HashMap<Object, OUT>(), false,
new KVMapCheckpointer<>(keyTypeInformation.createSerializer(executionConfig),
outTypeSerializer));
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,33 +17,48 @@

package org.apache.flink.streaming.api.operators;

import java.util.HashMap;

import org.apache.flink.api.common.functions.ReduceFunction;
import org.apache.flink.api.common.state.OperatorState;
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.state.KVMapCheckpointer;
import org.apache.flink.streaming.api.watermark.Watermark;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;

import java.util.HashMap;

public class StreamGroupedReduce<IN> extends AbstractUdfStreamOperator<IN, ReduceFunction<IN>>
implements OneInputStreamOperator<IN, IN>{
implements OneInputStreamOperator<IN, IN> {

private static final long serialVersionUID = 1L;

private KeySelector<IN, ?> keySelector;
private transient OperatorState<HashMap<Object, IN>> values;

public StreamGroupedReduce(ReduceFunction<IN> reducer, KeySelector<IN, ?> keySelector) {
// Store the typeinfo, create serializer during runtime
private TypeInformation<Object> keyTypeInformation;
private TypeInformation<IN> valueTypeInformation;

@SuppressWarnings("unchecked")
public StreamGroupedReduce(ReduceFunction<IN> reducer, KeySelector<IN, ?> keySelector,
TypeInformation<IN> typeInformation) {
super(reducer);
this.keySelector = keySelector;
valueTypeInformation = typeInformation;
keyTypeInformation = (TypeInformation<Object>) TypeExtractor
.getKeySelectorTypes(keySelector, typeInformation);
}

@Override
public void open(Configuration parameters) throws Exception {
super.open(parameters);

values = runtimeContext.getOperatorState("flink_internal_reduce_values",
new HashMap<Object, IN>(), false);
new HashMap<Object, IN>(), false,
new KVMapCheckpointer<>(keyTypeInformation.createSerializer(executionConfig),
valueTypeInformation.createSerializer(executionConfig)));
}

@Override
Expand All @@ -67,4 +82,5 @@ public void processWatermark(Watermark mark) throws Exception {
output.emitWatermark(mark);
}


}
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
/*
* 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.streaming.api.state;

import org.apache.flink.api.common.state.StateCheckpointer;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.typeutils.runtime.ByteArrayInputView;
import org.apache.flink.core.memory.DataOutputView;
import org.apache.flink.core.memory.OutputViewDataOutputStreamWrapper;

import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;

/**
* Implementation of the {@link StateCheckpointer} interface for a map storing
* types compatible with Flink's serialization system.
*
* @param <K> key type
* @param <V> value type
*/
public class KVMapCheckpointer<K, V> implements StateCheckpointer<HashMap<K, V>, byte[]> {

private TypeSerializer<K> keySerializer;
private TypeSerializer<V> valueSerializer;

public KVMapCheckpointer(TypeSerializer<K> keySerializer, TypeSerializer<V> valueSerializer) {
this.keySerializer = keySerializer;
this.valueSerializer = valueSerializer;
}

@Override
public byte[] snapshotState(HashMap<K, V> stateMap, long checkpointId, long checkpointTimestamp) {
ByteArrayOutputStream bos = new ByteArrayOutputStream(stateMap.size() * 16);
DataOutputView out = new OutputViewDataOutputStreamWrapper(new DataOutputStream(bos));
try {
out.writeInt(stateMap.size());
for (Map.Entry<K, V> kv : stateMap.entrySet()) {
keySerializer.serialize(kv.getKey(), out);
valueSerializer.serialize(kv.getValue(), out);
}
} catch (IOException e) {
throw new RuntimeException("Failed to write snapshot", e);
}
return bos.toByteArray();
}

@Override
public HashMap<K, V> restoreState(byte[] stateSnapshot) {
ByteArrayInputView in = new ByteArrayInputView(stateSnapshot);

HashMap<K, V> returnMap = new HashMap<>();
try {
int size = in.readInt();
for (int i = 0; i < size; i++) {
returnMap.put(keySerializer.deserialize(in), valueSerializer.deserialize(in));
}
} catch (IOException e) {
throw new RuntimeException("Failed to read snapshot", e);
}

return returnMap;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -92,15 +92,15 @@ public void groupSumIntegerTest() {
1, typeInfo, AggregationType.MAX, config);

List<Tuple2<Integer, Integer>> groupedSumList = MockContext.createAndExecute(
new StreamGroupedReduce<>(sumFunction, keySelector),
new StreamGroupedReduce<>(sumFunction, keySelector, typeInfo),
getInputList());

List<Tuple2<Integer, Integer>> groupedMinList = MockContext.createAndExecute(
new StreamGroupedReduce<>(minFunction, keySelector),
new StreamGroupedReduce<>(minFunction, keySelector, typeInfo),
getInputList());

List<Tuple2<Integer, Integer>> groupedMaxList = MockContext.createAndExecute(
new StreamGroupedReduce<>(maxFunction, keySelector),
new StreamGroupedReduce<>(maxFunction, keySelector, typeInfo),
getInputList());

assertEquals(expectedGroupSumList, groupedSumList);
Expand Down Expand Up @@ -156,13 +156,13 @@ public void pojoGroupSumIntegerTest() {
false, config);

List<MyPojo> groupedSumList = MockContext.createAndExecute(
new StreamGroupedReduce<>(sumFunction, keySelector),
new StreamGroupedReduce<>(sumFunction, keySelector, typeInfo),
getInputPojoList());
List<MyPojo> groupedMinList = MockContext.createAndExecute(
new StreamGroupedReduce<>(minFunction, keySelector),
new StreamGroupedReduce<>(minFunction, keySelector, typeInfo),
getInputPojoList());
List<MyPojo> groupedMaxList = MockContext.createAndExecute(
new StreamGroupedReduce<>(maxFunction, keySelector),
new StreamGroupedReduce<>(maxFunction, keySelector, typeInfo),
getInputPojoList());

assertEquals(expectedGroupSumList, groupedSumList);
Expand Down Expand Up @@ -216,16 +216,16 @@ public void minMaxByTest() {
new ComparableAggregator<>(1, typeInfo, AggregationType.MINBY, false, config);

assertEquals(maxByFirstExpected, MockContext.createAndExecute(
new StreamGroupedReduce<>(maxByFunctionFirst, keySelector),
new StreamGroupedReduce<>(maxByFunctionFirst, keySelector, typeInfo),
getInputByList()));
assertEquals(maxByLastExpected, MockContext.createAndExecute(
new StreamGroupedReduce<>(maxByFunctionLast, keySelector),
new StreamGroupedReduce<>(maxByFunctionLast, keySelector, typeInfo),
getInputByList()));
assertEquals(minByLastExpected, MockContext.createAndExecute(
new StreamGroupedReduce<>(minByFunctionLast, keySelector),
new StreamGroupedReduce<>(minByFunctionLast, keySelector, typeInfo),
getInputByList()));
assertEquals(minByFirstExpected, MockContext.createAndExecute(
new StreamGroupedReduce<>(minByFunctionFirst, keySelector),
new StreamGroupedReduce<>(minByFunctionFirst, keySelector, typeInfo),
getInputByList()));
}

Expand Down Expand Up @@ -274,16 +274,16 @@ public void pojoMinMaxByTest() {
new ComparableAggregator<>("f1", typeInfo, AggregationType.MINBY, false, config);

assertEquals(maxByFirstExpected, MockContext.createAndExecute(
new StreamGroupedReduce<>(maxByFunctionFirst, keySelector),
new StreamGroupedReduce<>(maxByFunctionFirst, keySelector, typeInfo),
getInputByPojoList()));
assertEquals(maxByLastExpected, MockContext.createAndExecute(
new StreamGroupedReduce<>(maxByFunctionLast, keySelector),
new StreamGroupedReduce<>(maxByFunctionLast, keySelector, typeInfo),
getInputByPojoList()));
assertEquals(minByLastExpected, MockContext.createAndExecute(
new StreamGroupedReduce<>(minByFunctionLast, keySelector),
new StreamGroupedReduce<>(minByFunctionLast, keySelector, typeInfo),
getInputByPojoList()));
assertEquals(minByFirstExpected, MockContext.createAndExecute(
new StreamGroupedReduce<>(minByFunctionFirst, keySelector),
new StreamGroupedReduce<>(minByFunctionFirst, keySelector, typeInfo),
getInputByPojoList()));
}

Expand Down
Loading

0 comments on commit c414ea9

Please sign in to comment.