/** * 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 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(AggregationFunction)} instead */ @Deprecated public <R> SingleOutputStreamOperator<R> fold(R initialValue, FoldFunction<T, R> function) { if (function instanceof RichFunction) { throw new UnsupportedOperationException("FoldFunction can not be a RichFunction. " + "Please use fold(FoldFunction, WindowFunction) instead."); } TypeInformation<R> resultType = TypeExtractor.getFoldReturnTypes(function, input.getType(), Utils.getCallLocationName(), true); return fold(initialValue, function, resultType); }
/** * 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 fold function. * * @param initialValue The initial value of the fold. * @param foldFunction The fold function that is used for incremental aggregation. * @param windowFunction The window function. * @return The data stream that is the result of applying the window function to the window. * * @deprecated use {@link #aggregate(AggregateFunction, WindowFunction)} instead */ @PublicEvolving @Deprecated public <R, ACC> SingleOutputStreamOperator<R> fold(ACC initialValue, FoldFunction<T, ACC> foldFunction, ProcessWindowFunction<ACC, R, K, W> windowFunction) { if (foldFunction instanceof RichFunction) { throw new UnsupportedOperationException("FoldFunction can not be a RichFunction."); } TypeInformation<ACC> foldResultType = TypeExtractor.getFoldReturnTypes(foldFunction, input.getType(), Utils.getCallLocationName(), true); TypeInformation<R> windowResultType = getProcessWindowFunctionReturnType(windowFunction, foldResultType, Utils.getCallLocationName()); return fold(initialValue, foldFunction, windowFunction, foldResultType, windowResultType); }
/** * 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 fold function. * * @param initialValue The initial value of the fold. * @param foldFunction The fold function that is used for incremental aggregation. * @param function The window function. * @return The data stream that is the result of applying the window function to the window. * * @deprecated use {@link #aggregate(AggregateFunction, WindowFunction)} instead */ @PublicEvolving @Deprecated public <ACC, R> SingleOutputStreamOperator<R> fold(ACC initialValue, FoldFunction<T, ACC> foldFunction, WindowFunction<ACC, R, K, W> function) { TypeInformation<ACC> foldAccumulatorType = TypeExtractor.getFoldReturnTypes(foldFunction, input.getType(), Utils.getCallLocationName(), true); TypeInformation<R> resultType = getWindowFunctionReturnType(function, foldAccumulatorType); return fold(initialValue, foldFunction, function, foldAccumulatorType, resultType); }
@Test public void testFoldWindowState() throws Exception { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); env.registerTypeWithKryoSerializer(File.class, JavaSerializer.class); DataStream<String> src = env.fromElements("abc"); SingleOutputStreamOperator<?> result = src .keyBy(new KeySelector<String, String>() { @Override public String getKey(String value) { return null; } }) .timeWindow(Time.milliseconds(1000)) .fold(new File("/"), new FoldFunction<String, File>() { @Override public File fold(File a, String e) { return null; } }); validateStateDescriptorConfigured(result); }
.keyBy(0) .window(TumblingEventTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS))) .fold(Tuple2.of("R:", 0), new FoldFunction<Tuple2<String, Integer>, Tuple2<String, Integer>>() { @Override public Tuple2<String, Integer> fold(Tuple2<String, Integer> accumulator,
@Test public void testSessionWithFoldFails() throws Exception { // verify that fold does not work with merging windows StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); WindowedStream<String, String, TimeWindow> windowedStream = env.fromElements("Hello", "Ciao") .keyBy(new KeySelector<String, String>() { @Override public String getKey(String value) throws Exception { return value; } }) .window(EventTimeSessionWindows.withGap(Time.seconds(5))); try { windowedStream.fold("", new FoldFunction<String, String>() { private static final long serialVersionUID = -4567902917104921706L; @Override public String fold(String accumulator, String value) throws Exception { return accumulator; } }); } catch (UnsupportedOperationException e) { // expected // use a catch to ensure that the exception is thrown by the fold return; } fail("The fold call should fail."); }
.keyBy(0) .window(TumblingEventTimeWindows.of(Time.of(3, TimeUnit.MILLISECONDS))) .fold(Tuple2.of(0, "R:"), new FoldFunction<Tuple2<String, Integer>, Tuple2<Integer, String>>() { @Override public Tuple2<Integer, String> fold(Tuple2<Integer, String> accumulator, Tuple2<String, Integer> value) throws Exception {
/** * .fold() does not support RichFoldFunction, since the fold function is used internally * in a {@code FoldingState}. */ @Test(expected = UnsupportedOperationException.class) public void testFoldWithRichFolderFails() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2)); env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime); source .keyBy(0) .window(SlidingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS))) .fold(new Tuple2<>("", 0), new RichFoldFunction<Tuple2<String, Integer>, Tuple2<String, Integer>>() { @Override public Tuple2<String, Integer> fold(Tuple2<String, Integer> value1, Tuple2<String, Integer> value2) throws Exception { return null; } }); fail("exception was not thrown"); }
@Test @SuppressWarnings("rawtypes") public void testFoldEventTimeWindows() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); DataStream<Tuple2<String, Integer>> source = env.fromElements( Tuple2.of("hello", 1), Tuple2.of("hello", 2)); DataStream<Tuple2<String, Integer>> window1 = source .keyBy(0) .timeWindow(Time.of(1000, TimeUnit.MILLISECONDS), Time.of(100, TimeUnit.MILLISECONDS)) .fold(new Tuple2<>("", 1), new DummyFolder()); OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform1 = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation(); OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<String, Integer>> operator1 = transform1.getOperator(); Assert.assertTrue(operator1 instanceof WindowOperator); WindowOperator winOperator1 = (WindowOperator) operator1; Assert.assertTrue(winOperator1.getTrigger() instanceof EventTimeTrigger); Assert.assertTrue(winOperator1.getWindowAssigner() instanceof SlidingEventTimeWindows); Assert.assertTrue(winOperator1.getStateDescriptor() instanceof FoldingStateDescriptor); }
@Test @SuppressWarnings("rawtypes") public void testFoldEventTime() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2)); DataStream<Tuple3<String, String, Integer>> window1 = source .keyBy(0) .window(SlidingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS))) .fold(new Tuple3<>("", "", 1), new DummyFolder()); OneInputTransformation<Tuple2<String, Integer>, Tuple3<String, String, Integer>> transform = (OneInputTransformation<Tuple2<String, Integer>, Tuple3<String, String, Integer>>) window1.getTransformation(); OneInputStreamOperator<Tuple2<String, Integer>, Tuple3<String, String, Integer>> operator = transform.getOperator(); Assert.assertTrue(operator instanceof WindowOperator); WindowOperator<String, Tuple2<String, Integer>, ?, ?, ?> winOperator = (WindowOperator<String, Tuple2<String, Integer>, ?, ?, ?>) operator; Assert.assertTrue(winOperator.getTrigger() instanceof EventTimeTrigger); Assert.assertTrue(winOperator.getWindowAssigner() instanceof SlidingEventTimeWindows); Assert.assertTrue(winOperator.getStateDescriptor() instanceof FoldingStateDescriptor); processElementAndEnsureOutput(winOperator, winOperator.getKeySelector(), BasicTypeInfo.STRING_TYPE_INFO, new Tuple2<>("hello", 1)); }
@Test @SuppressWarnings("rawtypes") public void testFoldProcessingTime() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime); DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2)); DataStream<Tuple3<String, String, Integer>> window = source .keyBy(new TupleKeySelector()) .window(SlidingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS))) .fold(new Tuple3<>("", "", 0), new DummyFolder()); OneInputTransformation<Tuple2<String, Integer>, Tuple3<String, String, Integer>> transform = (OneInputTransformation<Tuple2<String, Integer>, Tuple3<String, String, Integer>>) window.getTransformation(); OneInputStreamOperator<Tuple2<String, Integer>, Tuple3<String, String, Integer>> operator = transform.getOperator(); Assert.assertTrue(operator instanceof WindowOperator); WindowOperator<String, Tuple2<String, Integer>, ?, ?, ?> winOperator = (WindowOperator<String, Tuple2<String, Integer>, ?, ?, ?>) operator; Assert.assertTrue(winOperator.getTrigger() instanceof ProcessingTimeTrigger); Assert.assertTrue(winOperator.getWindowAssigner() instanceof SlidingProcessingTimeWindows); Assert.assertTrue(winOperator.getStateDescriptor() instanceof FoldingStateDescriptor); processElementAndEnsureOutput(winOperator, winOperator.getKeySelector(), BasicTypeInfo.STRING_TYPE_INFO, new Tuple2<>("hello", 1)); }
@Test @SuppressWarnings("rawtypes") public void testFoldWithCustomTrigger() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2)); DataStream<Tuple3<String, String, Integer>> window1 = source .keyBy(0) .window(SlidingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS))) .trigger(CountTrigger.of(1)) .fold(new Tuple3<>("", "", 1), new DummyFolder()); OneInputTransformation<Tuple2<String, Integer>, Tuple3<String, String, Integer>> transform = (OneInputTransformation<Tuple2<String, Integer>, Tuple3<String, String, Integer>>) window1.getTransformation(); OneInputStreamOperator<Tuple2<String, Integer>, Tuple3<String, String, Integer>> operator = transform.getOperator(); Assert.assertTrue(operator instanceof WindowOperator); WindowOperator<String, Tuple2<String, Integer>, ?, ?, ?> winOperator = (WindowOperator<String, Tuple2<String, Integer>, ?, ?, ?>) operator; Assert.assertTrue(winOperator.getTrigger() instanceof CountTrigger); Assert.assertTrue(winOperator.getWindowAssigner() instanceof SlidingEventTimeWindows); Assert.assertTrue(winOperator.getStateDescriptor() instanceof FoldingStateDescriptor); processElementAndEnsureOutput(winOperator, winOperator.getKeySelector(), BasicTypeInfo.STRING_TYPE_INFO, new Tuple2<>("hello", 1)); }
.keyBy(new TupleKeySelector()) .window(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS))) .fold(new Tuple3<>("", "empty", 0), new DummyFolder(), new ProcessWindowFunction<Tuple3<String, String, Integer>, Tuple2<String, Integer>, String, TimeWindow>() { private static final long serialVersionUID = 1L;
.keyBy(new TupleKeySelector()) .window(TumblingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS))) .fold(new Tuple3<>("", "", 0), new DummyFolder(), new ProcessWindowFunction<Tuple3<String, String, Integer>, Tuple2<String, Integer>, String, TimeWindow>() { private static final long serialVersionUID = 1L;
.keyBy(new TupleKeySelector()) .window(TumblingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS))) .fold(new Tuple3<>("", "", 0), new DummyFolder(), new WindowFunction<Tuple3<String, String, Integer>, Tuple2<String, Integer>, String, TimeWindow>() { private static final long serialVersionUID = 1L;
.keyBy(new TupleKeySelector()) .window(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS))) .fold(new Tuple3<>("", "empty", 0), new DummyFolder(), new WindowFunction<Tuple3<String, String, Integer>, Tuple2<String, Integer>, String, TimeWindow>() { private static final long serialVersionUID = 1L;
@Test @SuppressWarnings({"rawtypes", "unchecked"}) public void testFoldWithEvictor() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); DataStream<Tuple2<String, Integer>> source = env.fromElements(Tuple2.of("hello", 1), Tuple2.of("hello", 2)); DataStream<Tuple3<String, String, Integer>> window1 = source .keyBy(0) .window(SlidingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS))) .evictor(CountEvictor.of(100)) .fold(new Tuple3<>("", "", 1), new DummyFolder()); OneInputTransformation<Tuple2<String, Integer>, Tuple3<String, String, Integer>> transform = (OneInputTransformation<Tuple2<String, Integer>, Tuple3<String, String, Integer>>) window1.getTransformation(); OneInputStreamOperator<Tuple2<String, Integer>, Tuple3<String, String, Integer>> operator = transform.getOperator(); Assert.assertTrue(operator instanceof EvictingWindowOperator); EvictingWindowOperator<String, Tuple2<String, Integer>, ?, ?> winOperator = (EvictingWindowOperator<String, Tuple2<String, Integer>, ?, ?>) operator; Assert.assertTrue(winOperator.getTrigger() instanceof EventTimeTrigger); Assert.assertTrue(winOperator.getEvictor() instanceof CountEvictor); Assert.assertTrue(winOperator.getWindowAssigner() instanceof SlidingEventTimeWindows); Assert.assertTrue(winOperator.getStateDescriptor() instanceof ListStateDescriptor); winOperator.setOutputType((TypeInformation) window1.getType(), new ExecutionConfig()); processElementAndEnsureOutput(winOperator, winOperator.getKeySelector(), BasicTypeInfo.STRING_TYPE_INFO, new Tuple2<>("hello", 1)); }
.window(SlidingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS), Time.of(100, TimeUnit.MILLISECONDS))) .evictor(CountEvictor.of(100)) .fold( new Tuple3<>("", "", 1), new DummyFolder(),
/** * Performs a neighborhood fold on the graph window stream. * * @param initialValue * @param foldFunction * @return the result stream after applying the user-defined fold operation on the window */ public <T> DataStream<T> foldNeighbors(T initialValue, final EdgesFold<K, EV, T> foldFunction) { return windowedStream.fold(initialValue, new EdgesFoldFunction<K, EV, T>(foldFunction)); }