/** * Applies the given fold function to each 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. * * @param function The fold function. * @return The data stream that is the result of applying the fold function to the window. * * @deprecated use {@link #aggregate(AggregateFunction, TypeInformation, TypeInformation)} instead */ @Deprecated public <R> SingleOutputStreamOperator<R> fold(R initialValue, FoldFunction<T, R> function, TypeInformation<R> resultType) { if (function instanceof RichFunction) { throw new UnsupportedOperationException("FoldFunction can not be a RichFunction. " + "Please use fold(FoldFunction, WindowFunction) instead."); } return fold(initialValue, function, new PassThroughWindowFunction<K, W, R>(), resultType, 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. * * <p>This window will try and incrementally aggregate data as much as the window policies * permit. For example, tumbling time windows can aggregate the data, meaning that only one * element per key is stored. Sliding time windows will aggregate on the granularity of the * slide interval, so a few elements are stored per key (one per slide interval). * Custom windows may not be able to incrementally aggregate, or may need to store extra values * in an aggregation tree. * * @param function The reduce function. * @return The data stream that is the result of applying the reduce function to the window. */ @SuppressWarnings("unchecked") public SingleOutputStreamOperator<T> reduce(ReduceFunction<T> function) { if (function instanceof RichFunction) { throw new UnsupportedOperationException("ReduceFunction of reduce can not be a RichFunction. " + "Please use reduce(ReduceFunction, WindowFunction) instead."); } //clean the closure function = input.getExecutionEnvironment().clean(function); return reduce(function, new PassThroughWindowFunction<K, W, T>()); }
/** * Applies the given aggregation function to each window. The aggregation function is called for * each element, aggregating values incrementally and keeping the state to one accumulator * per key and window. * * @param function The aggregation function. * @return The data stream that is the result of applying the aggregation function to the window. * * @param <ACC> The type of the AggregateFunction's accumulator * @param <R> The type of the elements in the resulting stream, equal to the * AggregateFunction's result type */ @PublicEvolving public <ACC, R> SingleOutputStreamOperator<R> aggregate( AggregateFunction<T, ACC, R> function, TypeInformation<ACC> accumulatorType, TypeInformation<R> resultType) { checkNotNull(function, "function"); checkNotNull(accumulatorType, "accumulatorType"); checkNotNull(resultType, "resultType"); if (function instanceof RichFunction) { throw new UnsupportedOperationException("This aggregation function cannot be a RichFunction."); } return aggregate(function, new PassThroughWindowFunction<K, W, R>(), accumulatorType, resultType); }
@Test @SuppressWarnings("unchecked") public void testTumblingEventTimeWindowsReduce() throws Exception { closeCalled.set(0); final int windowSize = 3; ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents", new SumReducer(), STRING_INT_TUPLE.createSerializer(new ExecutionConfig())); WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>( TumblingEventTimeWindows.of(Time.of(windowSize, TimeUnit.SECONDS)), new TimeWindow.Serializer(), new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()), EventTimeTrigger.create(), 0, null /* late data output tag */); testTumblingEventTimeWindows(operator); }
@Test @SuppressWarnings("unchecked") public void testSlidingEventTimeWindowsReduce() throws Exception { closeCalled.set(0); final int windowSize = 3; final int windowSlide = 1; ReducingStateDescriptor<Tuple2<String, Integer>> stateDesc = new ReducingStateDescriptor<>("window-contents", new SumReducer(), STRING_INT_TUPLE.createSerializer(new ExecutionConfig())); WindowOperator<String, Tuple2<String, Integer>, Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow> operator = new WindowOperator<>( SlidingEventTimeWindows.of(Time.of(windowSize, TimeUnit.SECONDS), Time.of(windowSlide, TimeUnit.SECONDS)), new TimeWindow.Serializer(), new TupleKeySelector(), BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()), EventTimeTrigger.create(), 0, null /* late data output tag */); testSlidingEventTimeWindows(operator); }
BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()), EventTimeTrigger.create(), 0,
BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()), EventTimeTrigger.create(), lateness,
keySerializer, stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<>()), EventTimeTrigger.create(), 0,
BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()), ProcessingTimeTrigger.create(), 0,
BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()), PurgingTrigger.of(EventTimeTrigger.create()), lateness,
BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()), EventTimeTrigger.create(), lateness,
BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()), EventTimeTrigger.create(), lateness,
BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()), ProcessingTimeTrigger.create(), 0,
BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()), EventTimeTrigger.create(), lateness,
BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>()), ContinuousEventTimeTrigger.of(Time.of(windowSize, TimeUnit.SECONDS)), 0,
BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()), ProcessingTimeTrigger.create(), 0,
BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()), ProcessingTimeTrigger.create(), 0,
BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, TimeWindow, Tuple2<String, Integer>>()), ProcessingTimeTrigger.create(), 0,
BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>()), PurgingTrigger.of(CountTrigger.of(windowSize)), 0, BasicTypeInfo.STRING_TYPE_INFO.createSerializer(new ExecutionConfig()), stateDesc, new InternalSingleValueWindowFunction<>(new PassThroughWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>()), PurgingTrigger.of(CountTrigger.of(windowSize)), 0,
new SumReducer(), new PassThroughWindowFunction<String, GlobalWindow, Tuple2<String, Integer>>())), CountTrigger.of(windowSlide), CountEvictor.of(windowSize),