private DataStream<Tuple> processInput( String boltId, IRichBolt userBolt, GlobalStreamId streamId, Grouping grouping, Map<String, DataStream<Tuple>> producer) { assert (userBolt != null); assert (boltId != null); assert (streamId != null); assert (grouping != null); assert (producer != null); final String producerId = streamId.get_componentId(); final String inputStreamId = streamId.get_streamId(); DataStream<Tuple> inputStream = producer.get(inputStreamId); final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer(); declarers.put(boltId, declarer); userBolt.declareOutputFields(declarer); this.outputStreams.put(boltId, declarer.outputStreams); // if producer was processed already if (grouping.is_set_shuffle()) { // Storm uses a round-robin shuffle strategy inputStream = inputStream.rebalance(); } else if (grouping.is_set_fields()) { // global grouping is emulated in Storm via an empty fields grouping list final List<String> fields = grouping.get_fields(); if (fields.size() > 0) { FlinkOutputFieldsDeclarer prodDeclarer = this.declarers.get(producerId); inputStream = inputStream.keyBy( prodDeclarer.getGroupingFieldIndexes(inputStreamId, grouping.get_fields())); } else { inputStream = inputStream.global(); } } else if (grouping.is_set_all()) { inputStream = inputStream.broadcast(); } else if (!grouping.is_set_local_or_shuffle()) { throw new UnsupportedOperationException( "Flink only supports (local-or-)shuffle, fields, all, and global grouping"); } return inputStream; }
/** * Tests union functionality. This ensures that self-unions and unions of streams with differing * parallelism work. * * @throws Exception */ @Test public void testUnion() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(4); DataStream<Long> input1 = env.generateSequence(0, 0) .map( new MapFunction<Long, Long>() { @Override public Long map(Long value) throws Exception { return null; } }); DataStream<Long> selfUnion = input1 .union(input1) .map( new MapFunction<Long, Long>() { @Override public Long map(Long value) throws Exception { return null; } }); DataStream<Long> input6 = env.generateSequence(0, 0) .map( new MapFunction<Long, Long>() { @Override public Long map(Long value) throws Exception { return null; } }); DataStream<Long> selfUnionDifferentPartition = input6 .broadcast() .union(input6) .map( new MapFunction<Long, Long>() { @Override public Long map(Long value) throws Exception { return null; } }); DataStream<Long> input2 = env.generateSequence(0, 0) .map( new MapFunction<Long, Long>() { @Override public Long map(Long value) throws Exception { return null; } }) .setParallelism(4); DataStream<Long> input3 = env.generateSequence(0, 0) .map( new MapFunction<Long, Long>() { @Override public Long map(Long value) throws Exception { return null; } }) .setParallelism(2); DataStream<Long> unionDifferingParallelism = input2 .union(input3) .map( new MapFunction<Long, Long>() { @Override public Long map(Long value) throws Exception { return null; } }) .setParallelism(4); DataStream<Long> input4 = env.generateSequence(0, 0) .map( new MapFunction<Long, Long>() { @Override public Long map(Long value) throws Exception { return null; } }) .setParallelism(2); DataStream<Long> input5 = env.generateSequence(0, 0) .map( new MapFunction<Long, Long>() { @Override public Long map(Long value) throws Exception { return null; } }) .setParallelism(4); DataStream<Long> unionDifferingPartitioning = input4 .broadcast() .union(input5) .map( new MapFunction<Long, Long>() { @Override public Long map(Long value) throws Exception { return null; } }) .setParallelism(4); StreamGraph streamGraph = env.getStreamGraph(); // verify self union assertTrue(streamGraph.getStreamNode(selfUnion.getId()).getInEdges().size() == 2); for (StreamEdge edge : streamGraph.getStreamNode(selfUnion.getId()).getInEdges()) { assertTrue(edge.getPartitioner() instanceof ForwardPartitioner); } // verify self union with differnt partitioners assertTrue( streamGraph.getStreamNode(selfUnionDifferentPartition.getId()).getInEdges().size() == 2); boolean hasForward = false; boolean hasBroadcast = false; for (StreamEdge edge : streamGraph.getStreamNode(selfUnionDifferentPartition.getId()).getInEdges()) { if (edge.getPartitioner() instanceof ForwardPartitioner) { hasForward = true; } if (edge.getPartitioner() instanceof BroadcastPartitioner) { hasBroadcast = true; } } assertTrue(hasForward && hasBroadcast); // verify union of streams with differing parallelism assertTrue( streamGraph.getStreamNode(unionDifferingParallelism.getId()).getInEdges().size() == 2); for (StreamEdge edge : streamGraph.getStreamNode(unionDifferingParallelism.getId()).getInEdges()) { if (edge.getSourceId() == input2.getId()) { assertTrue(edge.getPartitioner() instanceof ForwardPartitioner); } else if (edge.getSourceId() == input3.getId()) { assertTrue(edge.getPartitioner() instanceof RebalancePartitioner); } else { fail("Wrong input edge."); } } // verify union of streams with differing partitionings assertTrue( streamGraph.getStreamNode(unionDifferingPartitioning.getId()).getInEdges().size() == 2); for (StreamEdge edge : streamGraph.getStreamNode(unionDifferingPartitioning.getId()).getInEdges()) { if (edge.getSourceId() == input4.getId()) { assertTrue(edge.getPartitioner() instanceof BroadcastPartitioner); } else if (edge.getSourceId() == input5.getId()) { assertTrue(edge.getPartitioner() instanceof ForwardPartitioner); } else { fail("Wrong input edge."); } } }