
在WindowedStream上可以执行,如reduce,aggregate,min,max等操作
关键是要理解windowOperator对KVState的运用,因为window是用它来存储window buffer的
采用不同的KVState,会有不同的效果,如ReduceState,ListState
Reduce
/**
* Applies the given window function to each window. The window function is called for each
* evaluation of the window for each key individually. The output of the window function is
* interpreted as a regular non-windowed stream.
*
* <p>
* Arriving data is incrementally aggregated using the given reducer.
*
* @param reduceFunction The reduce function that is used for incremental aggregation.
* @param function The window function.
* @param resultType Type information for the result type of the window function.
* @param legacyWindowOpType When migrating from an older Flink version, this flag indicates
* the type of the previous operator whose state we inherit.
* @return The data stream that is the result of applying the window function to the window.
*/
private <R> SingleOutputStreamOperator<R> reduce(
ReduceFunction<T> reduceFunction,
WindowFunction<T, R, K, W> function,
TypeInformation<R> resultType,
LegacyWindowOperatorType legacyWindowOpType) { String opName;
KeySelector<T, K> keySel = input.getKeySelector(); OneInputStreamOperator<T, R> operator; if (evictor != null) {
@SuppressWarnings({"unchecked", "rawtypes"})
TypeSerializer<StreamRecord<T>> streamRecordSerializer =
(TypeSerializer<StreamRecord<T>>) new StreamElementSerializer(input.getType().createSerializer(getExecutionEnvironment().getConfig())); ListStateDescriptor<StreamRecord<T>> stateDesc = //如果有evictor,这里state是list state,需要把windows整个cache下来,这样才能去evict
new ListStateDescriptor<>("window-contents", streamRecordSerializer); opName = "TriggerWindow(" + windowAssigner + ", " + stateDesc + ", " + trigger + ", " + evictor + ", " + udfName + ")"; //reduce的op name是这样拼的,可以看出window的所有相关配置 operator =
new EvictingWindowOperator<>(windowAssigner,
windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
keySel,
input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
stateDesc,
new InternalIterableWindowFunction<>(new ReduceApplyWindowFunction<>(reduceFunction, function)),
trigger,
evictor,
allowedLateness); } else { //如果没有evictor
ReducingStateDescriptor<T> stateDesc = new ReducingStateDescriptor<>("window-contents", //这里就是ReducingState,不需要cache整个list,所以效率更高
reduceFunction, //reduce的逻辑
input.getType().createSerializer(getExecutionEnvironment().getConfig())); opName = "TriggerWindow(" + windowAssigner + ", " + stateDesc + ", " + trigger + ", " + udfName + ")"; operator =
new WindowOperator<>(windowAssigner,
windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
keySel,
input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
stateDesc,
new InternalSingleValueWindowFunction<>(function),
trigger,
allowedLateness,
legacyWindowOpType);
} return input.transform(opName, resultType, operator);
}
reduceFunction,就是reduce的逻辑,一般只是指定这个参数
WindowFunction<T, R, K, W> function
TypeInformation<R> resultType
/**
* Applies a reduce function to the window. The window function is called for each evaluation
* of the window for each key individually. The output of the reduce function is interpreted
* as a regular non-windowed stream.
*/
这个function是WindowFunction,在window被fire时调用,resultType是WindowFunction的返回值,通过reduce,windowedStream会成为non-windowed stream
/**
* Emits the contents of the given window using the {@link InternalWindowFunction}.
*/
@SuppressWarnings("unchecked")
private void emitWindowContents(W window, ACC contents) throws Exception {
timestampedCollector.setAbsoluteTimestamp(window.maxTimestamp());
userFunction.apply(context.key, context.window, contents, timestampedCollector);
}
可以看到WindowFunction是对于每个key的window都会调用一遍
public void onEventTime(InternalTimer<K, W> timer) throws Exception { TriggerResult triggerResult = context.onEventTime(timer.getTimestamp());
if (triggerResult.isFire()) {
emitWindowContents(context.window, contents); //当window被fire的时候,调用
}
}
context.window是记录window的元数据,比如TimeWindow记录开始,结束时间
contents,是windowState,包含真正的数据
默认不指定,给定是
PassThroughWindowFunction
public class PassThroughWindowFunction<K, W extends Window, T> implements WindowFunction<T, T, K, W> { private static final long serialVersionUID = 1L; @Override
public void apply(K k, W window, Iterable<T> input, Collector<T> out) throws Exception {
for (T in: input) {
out.collect(in);
}
}
}
继续现在WindowOperator
@Override
public void processElement(StreamRecord<IN> element) throws Exception { for (W window: elementWindows) { //对于每个被assign的window // drop if the window is already late
if (isLate(window)) {
continue;
} windowState.setCurrentNamespace(window);
windowState.add(element.getValue()); //add element的值
windowState在WindowOperator.open中被初始化,
public void open() throws Exception {
// create (or restore) the state that hold the actual window contents
// NOTE - the state may be null in the case of the overriding evicting window operator
if (windowStateDescriptor != null) {
windowState = (InternalAppendingState<W, IN, ACC>) getOrCreateKeyedState(windowSerializer, windowStateDescriptor);
}
AbstractStreamOperator
protected <N, S extends State, T> S getOrCreateKeyedState(
TypeSerializer<N> namespaceSerializer,
StateDescriptor<S, T> stateDescriptor) throws Exception { if (keyedStateStore != null) {
return keyedStateBackend.getOrCreateKeyedState(namespaceSerializer, stateDescriptor);
}
AbstractKeyedStateBackend
public <N, S extends State, V> S getOrCreateKeyedState(
final TypeSerializer<N> namespaceSerializer,
StateDescriptor<S, V> stateDescriptor) throws Exception { // create a new blank key/value state
S state = stateDescriptor.bind(new StateBackend() {
@Override
public <T> ValueState<T> createValueState(ValueStateDescriptor<T> stateDesc) throws Exception {
return AbstractKeyedStateBackend.this.createValueState(namespaceSerializer, stateDesc);
} @Override
public <T> ListState<T> createListState(ListStateDescriptor<T> stateDesc) throws Exception {
return AbstractKeyedStateBackend.this.createListState(namespaceSerializer, stateDesc);
} @Override
public <T> ReducingState<T> createReducingState(ReducingStateDescriptor<T> stateDesc) throws Exception {
return AbstractKeyedStateBackend.this.createReducingState(namespaceSerializer, stateDesc);
} @Override
public <T, ACC, R> AggregatingState<T, R> createAggregatingState(
AggregatingStateDescriptor<T, ACC, R> stateDesc) throws Exception {
return AbstractKeyedStateBackend.this.createAggregatingState(namespaceSerializer, stateDesc);
}
可以看到这里根据不同的StateDescriptor调用bind,会生成不同的state
如果前面用的是ReducingStateDescriptor
@Override
public ReducingState<T> bind(StateBackend stateBackend) throws Exception {
return stateBackend.createReducingState(this);
}
所以如果用的是RockDB,
那么创建的是RocksDBReducingState
所以调用add的逻辑,
public class RocksDBReducingState<K, N, V>
extends AbstractRocksDBState<K, N, ReducingState<V>, ReducingStateDescriptor<V>, V>
implements InternalReducingState<N, V> {
@Override
public void add(V value) throws IOException {
try {
writeCurrentKeyWithGroupAndNamespace();
byte[] key = keySerializationStream.toByteArray();
byte[] valueBytes = backend.db.get(columnFamily, key); DataOutputViewStreamWrapper out = new DataOutputViewStreamWrapper(keySerializationStream);
if (valueBytes == null) {
keySerializationStream.reset();
valueSerializer.serialize(value, out);
backend.db.put(columnFamily, writeOptions, key, keySerializationStream.toByteArray());
} else {
V oldValue = valueSerializer.deserialize(new DataInputViewStreamWrapper(new ByteArrayInputStream(valueBytes)));
V newValue = reduceFunction.reduce(oldValue, value); //使用reduce函数合并value
keySerializationStream.reset();
valueSerializer.serialize(newValue, out);
backend.db.put(columnFamily, writeOptions, key, keySerializationStream.toByteArray()); //将新的value put到backend中
}
} catch (Exception e) {
throw new RuntimeException("Error while adding data to RocksDB", e);
}
}
aggregate
这里用AggregatingStateDescriptor
并且多个参数,TypeInformation<ACC> accumulatorType,因为aggregate是不断的更新这个accumulator
/**
* Applies the given window function to each window. The window function is called for each
* evaluation of the window for each key individually. The output of the window function is
* interpreted as a regular non-windowed stream.
*
* <p>Arriving data is incrementally aggregated using the given aggregate function. This means
* that the window function typically has only a single value to process when called.
*
* @param aggregateFunction The aggregation function that is used for incremental aggregation.
* @param windowFunction The window function.
* @param accumulatorType Type information for the internal accumulator type of the aggregation function
* @param resultType Type information for the result type of the window function
*
* @return The data stream that is the result of applying the window function to the window.
*
* @param <ACC> The type of the AggregateFunction's accumulator
* @param <V> The type of AggregateFunction's result, and the WindowFunction's input
* @param <R> The type of the elements in the resulting stream, equal to the
* WindowFunction's result type
*/
public <ACC, V, R> SingleOutputStreamOperator<R> aggregate(
AggregateFunction<T, ACC, V> aggregateFunction,
WindowFunction<V, R, K, W> windowFunction,
TypeInformation<ACC> accumulatorType,
TypeInformation<V> aggregateResultType,
TypeInformation<R> resultType) { if (evictor != null) { //evictor仍然是用ListState
} else {
AggregatingStateDescriptor<T, ACC, V> stateDesc = new AggregatingStateDescriptor<>("window-contents",
aggregateFunction, accumulatorType.createSerializer(getExecutionEnvironment().getConfig())); opName = "TriggerWindow(" + windowAssigner + ", " + stateDesc + ", " + trigger + ", " + udfName + ")"; operator = new WindowOperator<>(windowAssigner,
windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
keySel,
input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
stateDesc,
new InternalSingleValueWindowFunction<>(windowFunction),
trigger,
allowedLateness);
} return input.transform(opName, resultType, operator);
}
最终用到,
RocksDBAggregatingState
@Override
public R get() throws IOException {
try {
// prepare the current key and namespace for RocksDB lookup
writeCurrentKeyWithGroupAndNamespace();
final byte[] key = keySerializationStream.toByteArray(); // get the current value
final byte[] valueBytes = backend.db.get(columnFamily, key); if (valueBytes == null) {
return null;
} ACC accumulator = valueSerializer.deserialize(new DataInputViewStreamWrapper(new ByteArrayInputStreamWithPos(valueBytes)));
return aggFunction.getResult(accumulator); //返回accumulator的值
}
catch (IOException | RocksDBException e) {
throw new IOException("Error while retrieving value from RocksDB", e);
}
} @Override
public void add(T value) throws IOException {
try {
// prepare the current key and namespace for RocksDB lookup
writeCurrentKeyWithGroupAndNamespace();
final byte[] key = keySerializationStream.toByteArray();
keySerializationStream.reset(); // get the current value
final byte[] valueBytes = backend.db.get(columnFamily, key); // deserialize the current accumulator, or create a blank one
final ACC accumulator = valueBytes == null ? //create new或从state中反序列化出来
aggFunction.createAccumulator() :
valueSerializer.deserialize(new DataInputViewStreamWrapper(new ByteArrayInputStreamWithPos(valueBytes))); // aggregate the value into the accumulator
aggFunction.add(value, accumulator); //更新accumulator // serialize the new accumulator
final DataOutputViewStreamWrapper out = new DataOutputViewStreamWrapper(keySerializationStream);
valueSerializer.serialize(accumulator, out); // write the new value to RocksDB
backend.db.put(columnFamily, writeOptions, key, keySerializationStream.toByteArray());
}
catch (IOException | RocksDBException e) {
throw new IOException("Error while adding value to RocksDB", e);
}
}
给个aggFunction的例子,
private static class AddingFunction implements AggregateFunction<Long, MutableLong, Long> { @Override
public MutableLong createAccumulator() {
return new MutableLong();
} @Override
public void add(Long value, MutableLong accumulator) {
accumulator.value += value;
} @Override
public Long getResult(MutableLong accumulator) {
return accumulator.value;
} @Override
public MutableLong merge(MutableLong a, MutableLong b) {
a.value += b.value;
return a;
}
} private static final class MutableLong {
long value;
}
aggregate和reduce比,更通用,
reduce, A1 reduce A2 = A3
aggregate,a1 a2… aggregate = b
apply
更通用,就是不会再cache的时候做预算,而是需要cache整个windows数据,在触发的时候再apply
/**
* Applies the given window function to each window. The window function is called for each
* evaluation of the window for each key individually. The output of the window function is
* interpreted as a regular non-windowed stream.
*
* <p>
* Note that this function requires that all data in the windows is buffered until the window
* is evaluated, as the function provides no means of incremental aggregation.
*
* @param function The window function.
* @param resultType Type information for the result type of the window function
* @return The data stream that is the result of applying the window function to the window.
*/
public <R> SingleOutputStreamOperator<R> apply(WindowFunction<T, R, K, W> function, TypeInformation<R> resultType) { if (evictor != null) {
//
} else {
ListStateDescriptor<T> stateDesc = new ListStateDescriptor<>("window-contents", //因为要cache所有数据,所以一定是ListState
input.getType().createSerializer(getExecutionEnvironment().getConfig())); opName = "TriggerWindow(" + windowAssigner + ", " + stateDesc + ", " + trigger + ", " + udfName + ")"; operator =
new WindowOperator<>(windowAssigner,
windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()),
keySel,
input.getKeyType().createSerializer(getExecutionEnvironment().getConfig()),
stateDesc,
new InternalIterableWindowFunction<>(function),
trigger,
allowedLateness,
legacyWindowOpType);
} return input.transform(opName, resultType, operator);
}
这里就很简单了,你必须要给出WindowFunction,用于处理window触发时的结果
这里也需要指明resultType
而且使用ListStateDescriptor,这种state只是把element加到list中
AggregationFunction
如sum,min,max
/**
* Applies an aggregation that sums every window of the data stream at the
* given position.
*
* @param positionToSum The position in the tuple/array to sum
* @return The transformed DataStream.
*/
public SingleOutputStreamOperator<T> sum(int positionToSum) {
return aggregate(new SumAggregator<>(positionToSum, input.getType(), input.getExecutionConfig()));
}
public class SumAggregator<T> extends AggregationFunction<T> {
public abstract class AggregationFunction<T> implements ReduceFunction<T> {
private static final long serialVersionUID = 1L; public enum AggregationType {
SUM, MIN, MAX, MINBY, MAXBY,
} }
可以看到,无法顾名思义,这些AggregationFunction,是用reduce实现的