private static Integer createDownStreamId(ConnectedDataStream dataStream) { SingleOutputStreamOperator coMap = dataStream.map( new CoMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Object>() { @Override public Object map1(Tuple2<Long, Long> value) { return null; } @Override public Object map2(Tuple2<Long, Long> value) { return null; } }); coMap.addSink(new NoOpSink()); return coMap.getId(); }
@SuppressWarnings("rawtypes,unchecked") private static Integer createDownStreamId(ConnectedStreams dataStream) { SingleOutputStreamOperator<?, ?> coMap = dataStream.map( new CoMapFunction<Tuple2<Long, Long>, Tuple2<Long, Long>, Object>() { private static final long serialVersionUID = 1L; @Override public Object map1(Tuple2<Long, Long> value) { return null; } @Override public Object map2(Tuple2<Long, Long> value) { return null; } }); coMap.addSink(new NoOpSink()); return coMap.getId(); }
/** Tests whether parallelism gets set. */ @Test public void testParallelism() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource<Tuple2<Long, Long>> src = env.fromElements(new Tuple2<>(0L, 0L)); env.setParallelism(10); SingleOutputStreamOperator<Long, ?> map = src.map( new MapFunction<Tuple2<Long, Long>, Long>() { @Override public Long map(Tuple2<Long, Long> value) throws Exception { return null; } }) .name("MyMap"); DataStream<Long> windowed = map.windowAll(GlobalWindows.create()) .trigger(PurgingTrigger.of(CountTrigger.of(10))) .fold( 0L, new FoldFunction<Long, Long>() { @Override public Long fold(Long accumulator, Long value) throws Exception { return null; } }); windowed.addSink(new NoOpSink<Long>()); DataStreamSink<Long> sink = map.addSink( new SinkFunction<Long>() { private static final long serialVersionUID = 1L; @Override public void invoke(Long value) throws Exception {} }); assertEquals(1, env.getStreamGraph().getStreamNode(src.getId()).getParallelism()); assertEquals(10, env.getStreamGraph().getStreamNode(map.getId()).getParallelism()); assertEquals(1, env.getStreamGraph().getStreamNode(windowed.getId()).getParallelism()); assertEquals( 10, env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getParallelism()); env.setParallelism(7); // Some parts, such as windowing rely on the fact that previous operators have a parallelism // set when instantiating the Discretizer. This would break if we dynamically changed // the parallelism of operations when changing the setting on the Execution Environment. assertEquals(1, env.getStreamGraph().getStreamNode(src.getId()).getParallelism()); assertEquals(10, env.getStreamGraph().getStreamNode(map.getId()).getParallelism()); assertEquals(1, env.getStreamGraph().getStreamNode(windowed.getId()).getParallelism()); assertEquals( 10, env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getParallelism()); try { src.setParallelism(3); fail(); } catch (IllegalArgumentException success) { // do nothing } DataStreamSource<Long> parallelSource = env.generateSequence(0, 0); parallelSource.addSink(new NoOpSink<Long>()); assertEquals(7, env.getStreamGraph().getStreamNode(parallelSource.getId()).getParallelism()); parallelSource.setParallelism(3); assertEquals(3, env.getStreamGraph().getStreamNode(parallelSource.getId()).getParallelism()); map.setParallelism(2); assertEquals(2, env.getStreamGraph().getStreamNode(map.getId()).getParallelism()); sink.setParallelism(4); assertEquals( 4, env.getStreamGraph().getStreamNode(sink.getTransformation().getId()).getParallelism()); }