@Test public void testProcessAllWindowState() throws Exception { final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setStreamTimeCharacteristic(TimeCharacteristic.IngestionTime); env.registerTypeWithKryoSerializer(File.class, JavaSerializer.class); DataStream<File> src = env.fromElements(new File("/")); SingleOutputStreamOperator<?> result = src .timeWindowAll(Time.milliseconds(1000)) .process(new ProcessAllWindowFunction<File, String, TimeWindow>() { @Override public void process(Context ctx, Iterable<File> input, Collector<String> out) {} }); validateListStateDescriptorConfigured(result); }
.assignTimestampsAndWatermarks(new TestWatermarkAssigner()) .timeWindowAll(Time.milliseconds(1), Time.milliseconds(1)) .process(new ProcessAllWindowFunction<Integer, Integer, TimeWindow>() { private static final long serialVersionUID = 1L;
@Test @SuppressWarnings("rawtypes") public void testProcessProcessingTime() 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<Tuple2<String, Integer>> window1 = source .windowAll(TumblingProcessingTimeWindows.of(Time.of(1, TimeUnit.SECONDS))) .process(new ProcessAllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() { private static final long serialVersionUID = 1L; @Override public void process( Context ctx, Iterable<Tuple2<String, Integer>> values, Collector<Tuple2<String, Integer>> out) throws Exception { for (Tuple2<String, Integer> in : values) { out.collect(in); } } }); OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation(); OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<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 TumblingProcessingTimeWindows); Assert.assertTrue(winOperator.getStateDescriptor() instanceof ListStateDescriptor); processElementAndEnsureOutput(winOperator, winOperator.getKeySelector(), BasicTypeInfo.STRING_TYPE_INFO, new Tuple2<>("hello", 1)); }
@Test @SuppressWarnings("rawtypes") public void testMergingWindowsWithEvictor() 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 .windowAll(EventTimeSessionWindows.withGap(Time.seconds(5))) .evictor(CountEvictor.of(5)) .process(new TestProcessAllWindowFunction()); 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 EventTimeSessionWindows); Assert.assertTrue(winOperator.getStateDescriptor() instanceof ListStateDescriptor); processElementAndEnsureOutput(winOperator, winOperator.getKeySelector(), BasicTypeInfo.STRING_TYPE_INFO, new Tuple2<>("hello", 1)); }
@Test @SuppressWarnings("rawtypes") public void testProcessEventTime() 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 .windowAll(TumblingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS))) .process(new ProcessAllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() { private static final long serialVersionUID = 1L; @Override public void process( Context ctx, Iterable<Tuple2<String, Integer>> values, Collector<Tuple2<String, Integer>> out) throws Exception { for (Tuple2<String, Integer> in : values) { out.collect(in); } } }); OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation(); OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<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 TumblingEventTimeWindows); Assert.assertTrue(winOperator.getStateDescriptor() instanceof ListStateDescriptor); processElementAndEnsureOutput(winOperator, winOperator.getKeySelector(), BasicTypeInfo.STRING_TYPE_INFO, new Tuple2<>("hello", 1)); }
@Test @SuppressWarnings("rawtypes") public void testProcessWithCustomTrigger() 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 .windowAll(TumblingEventTimeWindows.of(Time.of(1, TimeUnit.SECONDS))) .trigger(CountTrigger.of(1)) .process(new ProcessAllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() { private static final long serialVersionUID = 1L; @Override public void process( Context ctx, Iterable<Tuple2<String, Integer>> values, Collector<Tuple2<String, Integer>> out) throws Exception { for (Tuple2<String, Integer> in : values) { out.collect(in); } } }); OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>> transform = (OneInputTransformation<Tuple2<String, Integer>, Tuple2<String, Integer>>) window1.getTransformation(); OneInputStreamOperator<Tuple2<String, Integer>, Tuple2<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 TumblingEventTimeWindows); Assert.assertTrue(winOperator.getStateDescriptor() instanceof ListStateDescriptor); processElementAndEnsureOutput(winOperator, winOperator.getKeySelector(), BasicTypeInfo.STRING_TYPE_INFO, new Tuple2<>("hello", 1)); }
.trigger(CountTrigger.of(1)) .evictor(TimeEvictor.of(Time.of(100, TimeUnit.MILLISECONDS))) .process(new ProcessAllWindowFunction<Tuple2<String, Integer>, Tuple2<String, Integer>, TimeWindow>() { private static final long serialVersionUID = 1L;