@Test public void testConsecutiveSplitRejection() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource<Integer> src = env.fromElements(0, 0); OutputSelector<Integer> outputSelector = new DummyOutputSelector<>(); src.split(outputSelector).split(outputSelector).addSink(new DiscardingSink<>()); expectedException.expect(IllegalStateException.class); expectedException.expectMessage("Consecutive multiple splits are not supported. Splits are deprecated. Please use side-outputs."); env.getStreamGraph(); }
@Test public void testSplitAfterSideOutputRejection() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource<Integer> src = env.fromElements(0, 0); OutputTag<Integer> outputTag = new OutputTag<Integer>("dummy"){}; OutputSelector<Integer> outputSelector = new DummyOutputSelector<>(); src.getSideOutput(outputTag).split(outputSelector).addSink(new DiscardingSink<>()); expectedException.expect(IllegalStateException.class); expectedException.expectMessage("Split after side-outputs are not supported. Splits are deprecated. Please use side-outputs."); env.getStreamGraph(); }
@Test public void testSelectBetweenConsecutiveSplitRejection() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource<Integer> src = env.fromElements(0, 0); OutputSelector<Integer> outputSelector = new DummyOutputSelector<>(); src.split(outputSelector).select("dummy").split(outputSelector).addSink(new DiscardingSink<>()); expectedException.expect(IllegalStateException.class); expectedException.expectMessage("Consecutive multiple splits are not supported. Splits are deprecated. Please use side-outputs."); env.getStreamGraph(); }
@Test public void outputSelectorTest() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(3); TestListResultSink<Long> evenSink = new TestListResultSink<Long>(); TestListResultSink<Long> oddAndTenSink = new TestListResultSink<Long>(); TestListResultSink<Long> evenAndOddSink = new TestListResultSink<Long>(); TestListResultSink<Long> allSink = new TestListResultSink<Long>(); SplitStream<Long> source = env.generateSequence(1, 11).split(new MyOutputSelector()); source.select(EVEN).addSink(evenSink); source.select(ODD, TEN).addSink(oddAndTenSink); source.select(EVEN, ODD).addSink(evenAndOddSink); source.addSink(allSink); env.execute(); assertEquals(Arrays.asList(2L, 4L, 6L, 8L, 10L), evenSink.getSortedResult()); assertEquals(Arrays.asList(1L, 3L, 5L, 7L, 9L, 10L, 11L), oddAndTenSink.getSortedResult()); assertEquals(Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L, 11L), evenAndOddSink.getSortedResult()); assertEquals(Arrays.asList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L, 11L), allSink.getSortedResult()); } }
@Test public void testKeybyBetweenConsecutiveSplitRejection() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource<Integer> src = env.fromElements(0, 0); OutputSelector<Integer> outputSelector = new DummyOutputSelector<>(); src.split(outputSelector).select("dummy").keyBy(x -> x).split(outputSelector).addSink(new DiscardingSink<>()); expectedException.expect(IllegalStateException.class); expectedException.expectMessage("Consecutive multiple splits are not supported. Splits are deprecated. Please use side-outputs."); env.getStreamGraph(); }
@Test public void testUnionBetweenConsecutiveSplitRejection() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource<Integer> src = env.fromElements(0, 0); OutputSelector<Integer> outputSelector = new DummyOutputSelector<>(); src.split(outputSelector).select("dummy").union(src.map(x -> x)).split(outputSelector).addSink(new DiscardingSink<>()); expectedException.expect(IllegalStateException.class); expectedException.expectMessage("Consecutive multiple splits are not supported. Splits are deprecated. Please use side-outputs."); env.getStreamGraph(); }