/** Creates a Flink program that uses the specified spouts and bolts. */ private void translateTopology() { unprocessdInputsPerBolt.clear(); outputStreams.clear(); declarers.clear(); availableInputs.clear(); // Storm defaults to parallelism 1 env.setParallelism(1); /* Translation of topology */ for (final Entry<String, IRichSpout> spout : spouts.entrySet()) { final String spoutId = spout.getKey(); final IRichSpout userSpout = spout.getValue(); final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer(); userSpout.declareOutputFields(declarer); final HashMap<String, Fields> sourceStreams = declarer.outputStreams; this.outputStreams.put(spoutId, sourceStreams); declarers.put(spoutId, declarer); final HashMap<String, DataStream<Tuple>> outputStreams = new HashMap<String, DataStream<Tuple>>(); final DataStreamSource<?> source; if (sourceStreams.size() == 1) { final SpoutWrapper<Tuple> spoutWrapperSingleOutput = new SpoutWrapper<Tuple>(userSpout, spoutId, null, null); spoutWrapperSingleOutput.setStormTopology(stormTopology); final String outputStreamId = (String) sourceStreams.keySet().toArray()[0]; DataStreamSource<Tuple> src = env.addSource( spoutWrapperSingleOutput, spoutId, declarer.getOutputType(outputStreamId)); outputStreams.put(outputStreamId, src); source = src; } else { final SpoutWrapper<SplitStreamType<Tuple>> spoutWrapperMultipleOutputs = new SpoutWrapper<SplitStreamType<Tuple>>(userSpout, spoutId, null, null); spoutWrapperMultipleOutputs.setStormTopology(stormTopology); @SuppressWarnings({"unchecked", "rawtypes"}) DataStreamSource<SplitStreamType<Tuple>> multiSource = env.addSource( spoutWrapperMultipleOutputs, spoutId, (TypeInformation) TypeExtractor.getForClass(SplitStreamType.class)); SplitStream<SplitStreamType<Tuple>> splitSource = multiSource.split(new StormStreamSelector<Tuple>()); for (String streamId : sourceStreams.keySet()) { SingleOutputStreamOperator<Tuple, ?> outStream = splitSource.select(streamId).map(new SplitStreamMapper<Tuple>()); outStream.getTransformation().setOutputType(declarer.getOutputType(streamId)); outputStreams.put(streamId, outStream); } source = multiSource; } availableInputs.put(spoutId, outputStreams); final ComponentCommon common = stormTopology.get_spouts().get(spoutId).get_common(); if (common.is_set_parallelism_hint()) { int dop = common.get_parallelism_hint(); source.setParallelism(dop); } else { common.set_parallelism_hint(1); } } /** * 1. Connect all spout streams with bolts streams 2. Then proceed with the bolts stream already * connected * * <p>Because we do not know the order in which an iterator steps over a set, we might process a * consumer before its producer ->thus, we might need to repeat multiple times */ boolean makeProgress = true; while (bolts.size() > 0) { if (!makeProgress) { StringBuilder strBld = new StringBuilder(); strBld.append("Unable to build Topology. Could not connect the following bolts:"); for (String boltId : bolts.keySet()) { strBld.append("\n "); strBld.append(boltId); strBld.append(": missing input streams ["); for (Entry<GlobalStreamId, Grouping> streams : unprocessdInputsPerBolt.get(boltId)) { strBld.append("'"); strBld.append(streams.getKey().get_streamId()); strBld.append("' from '"); strBld.append(streams.getKey().get_componentId()); strBld.append("'; "); } strBld.append("]"); } throw new RuntimeException(strBld.toString()); } makeProgress = false; final Iterator<Entry<String, IRichBolt>> boltsIterator = bolts.entrySet().iterator(); while (boltsIterator.hasNext()) { final Entry<String, IRichBolt> bolt = boltsIterator.next(); final String boltId = bolt.getKey(); final IRichBolt userBolt = copyObject(bolt.getValue()); final ComponentCommon common = stormTopology.get_bolts().get(boltId).get_common(); Set<Entry<GlobalStreamId, Grouping>> unprocessedBoltInputs = unprocessdInputsPerBolt.get(boltId); if (unprocessedBoltInputs == null) { unprocessedBoltInputs = new HashSet<>(); unprocessedBoltInputs.addAll(common.get_inputs().entrySet()); unprocessdInputsPerBolt.put(boltId, unprocessedBoltInputs); } // check if all inputs are available final int numberOfInputs = unprocessedBoltInputs.size(); int inputsAvailable = 0; for (Entry<GlobalStreamId, Grouping> entry : unprocessedBoltInputs) { final String producerId = entry.getKey().get_componentId(); final String streamId = entry.getKey().get_streamId(); final HashMap<String, DataStream<Tuple>> streams = availableInputs.get(producerId); if (streams != null && streams.get(streamId) != null) { inputsAvailable++; } } if (inputsAvailable != numberOfInputs) { // traverse other bolts first until inputs are available continue; } else { makeProgress = true; boltsIterator.remove(); } final Map<GlobalStreamId, DataStream<Tuple>> inputStreams = new HashMap<>(numberOfInputs); for (Entry<GlobalStreamId, Grouping> input : unprocessedBoltInputs) { final GlobalStreamId streamId = input.getKey(); final Grouping grouping = input.getValue(); final String producerId = streamId.get_componentId(); final Map<String, DataStream<Tuple>> producer = availableInputs.get(producerId); inputStreams.put(streamId, processInput(boltId, userBolt, streamId, grouping, producer)); } final SingleOutputStreamOperator<?, ?> outputStream = createOutput(boltId, userBolt, inputStreams); if (common.is_set_parallelism_hint()) { int dop = common.get_parallelism_hint(); outputStream.setParallelism(dop); } else { common.set_parallelism_hint(1); } } } }
@Test public void operatorTest() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource<Long> src = env.generateSequence(0, 0); MapFunction<Long, Integer> mapFunction = new MapFunction<Long, Integer>() { @Override public Integer map(Long value) throws Exception { return null; } }; DataStream<Integer> map = src.map(mapFunction); map.addSink(new NoOpSink<Integer>()); assertEquals(mapFunction, getFunctionForDataStream(map)); FlatMapFunction<Long, Integer> flatMapFunction = new FlatMapFunction<Long, Integer>() { private static final long serialVersionUID = 1L; @Override public void flatMap(Long value, Collector<Integer> out) throws Exception {} }; DataStream<Integer> flatMap = src.flatMap(flatMapFunction); flatMap.addSink(new NoOpSink<Integer>()); assertEquals(flatMapFunction, getFunctionForDataStream(flatMap)); FilterFunction<Integer> filterFunction = new FilterFunction<Integer>() { @Override public boolean filter(Integer value) throws Exception { return false; } }; DataStream<Integer> unionFilter = map.union(flatMap).filter(filterFunction); unionFilter.addSink(new NoOpSink<Integer>()); assertEquals(filterFunction, getFunctionForDataStream(unionFilter)); try { env.getStreamGraph().getStreamEdge(map.getId(), unionFilter.getId()); } catch (RuntimeException e) { fail(e.getMessage()); } try { env.getStreamGraph().getStreamEdge(flatMap.getId(), unionFilter.getId()); } catch (RuntimeException e) { fail(e.getMessage()); } OutputSelector<Integer> outputSelector = new OutputSelector<Integer>() { @Override public Iterable<String> select(Integer value) { return null; } }; SplitStream<Integer> split = unionFilter.split(outputSelector); split.select("dummy").addSink(new NoOpSink<Integer>()); List<OutputSelector<?>> outputSelectors = env.getStreamGraph().getStreamNode(unionFilter.getId()).getOutputSelectors(); assertEquals(1, outputSelectors.size()); assertEquals(outputSelector, outputSelectors.get(0)); DataStream<Integer> select = split.select("a"); DataStreamSink<Integer> sink = select.print(); StreamEdge splitEdge = env.getStreamGraph().getStreamEdge(unionFilter.getId(), sink.getTransformation().getId()); assertEquals("a", splitEdge.getSelectedNames().get(0)); ConnectedStreams<Integer, Integer> connect = map.connect(flatMap); CoMapFunction<Integer, Integer, String> coMapper = new CoMapFunction<Integer, Integer, String>() { private static final long serialVersionUID = 1L; @Override public String map1(Integer value) { return null; } @Override public String map2(Integer value) { return null; } }; DataStream<String> coMap = connect.map(coMapper); coMap.addSink(new NoOpSink<String>()); assertEquals(coMapper, getFunctionForDataStream(coMap)); try { env.getStreamGraph().getStreamEdge(map.getId(), coMap.getId()); } catch (RuntimeException e) { fail(e.getMessage()); } try { env.getStreamGraph().getStreamEdge(flatMap.getId(), coMap.getId()); } catch (RuntimeException e) { fail(e.getMessage()); } }
@Test public void testChannelSelectors() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStreamSource<Long> src = env.generateSequence(0, 0); DataStream<Long> broadcast = src.broadcast(); DataStreamSink<Long> broadcastSink = broadcast.print(); StreamPartitioner<?> broadcastPartitioner = env.getStreamGraph() .getStreamEdge(src.getId(), broadcastSink.getTransformation().getId()) .getPartitioner(); assertTrue(broadcastPartitioner instanceof BroadcastPartitioner); DataStream<Long> shuffle = src.shuffle(); DataStreamSink<Long> shuffleSink = shuffle.print(); StreamPartitioner<?> shufflePartitioner = env.getStreamGraph() .getStreamEdge(src.getId(), shuffleSink.getTransformation().getId()) .getPartitioner(); assertTrue(shufflePartitioner instanceof ShufflePartitioner); DataStream<Long> forward = src.forward(); DataStreamSink<Long> forwardSink = forward.print(); StreamPartitioner<?> forwardPartitioner = env.getStreamGraph() .getStreamEdge(src.getId(), forwardSink.getTransformation().getId()) .getPartitioner(); assertTrue(forwardPartitioner instanceof ForwardPartitioner); DataStream<Long> rebalance = src.rebalance(); DataStreamSink<Long> rebalanceSink = rebalance.print(); StreamPartitioner<?> rebalancePartitioner = env.getStreamGraph() .getStreamEdge(src.getId(), rebalanceSink.getTransformation().getId()) .getPartitioner(); assertTrue(rebalancePartitioner instanceof RebalancePartitioner); DataStream<Long> global = src.global(); DataStreamSink<Long> globalSink = global.print(); StreamPartitioner<?> globalPartitioner = env.getStreamGraph() .getStreamEdge(src.getId(), globalSink.getTransformation().getId()) .getPartitioner(); assertTrue(globalPartitioner instanceof GlobalPartitioner); }
/** 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()); }