/** * Note: this test fails if we don't have the synchronized block in {@link * org.apache.flink.streaming.runtime.tasks.SourceStreamTask.SourceOutput} */ @Test public void testTwoInputOperatorWithoutChaining() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); DataStream<String> source = env.addSource(new InfiniteTestSource()); source .connect(source) .transform( "Custom Operator", BasicTypeInfo.STRING_TYPE_INFO, new TwoInputTimerOperator(StreamOperator.ChainingStrategy.NEVER)); boolean testSuccess = false; try { env.execute("Timer test"); } catch (JobExecutionException e) { if (e.getCause() instanceof TimerException) { TimerException te = (TimerException) e.getCause(); if (te.getCause() instanceof RuntimeException) { RuntimeException re = (RuntimeException) te.getCause(); if (re.getMessage().equals("TEST SUCCESS")) { testSuccess = true; } else { throw e; } } else { throw e; } } else { throw e; } } Assert.assertTrue(testSuccess); }
@SuppressWarnings({"unchecked", "rawtypes"}) private SingleOutputStreamOperator<?, ?> createOutput( String boltId, IRichBolt bolt, Map<GlobalStreamId, DataStream<Tuple>> inputStreams) { assert (boltId != null); assert (bolt != null); assert (inputStreams != null); Iterator<Entry<GlobalStreamId, DataStream<Tuple>>> iterator = inputStreams.entrySet().iterator(); Entry<GlobalStreamId, DataStream<Tuple>> input1 = iterator.next(); GlobalStreamId streamId1 = input1.getKey(); String inputStreamId1 = streamId1.get_streamId(); String inputComponentId1 = streamId1.get_componentId(); Fields inputSchema1 = this.outputStreams.get(inputComponentId1).get(inputStreamId1); DataStream<Tuple> singleInputStream = input1.getValue(); DataStream<StormTuple<Tuple>> mergedInputStream = null; while (iterator.hasNext()) { Entry<GlobalStreamId, DataStream<Tuple>> input2 = iterator.next(); GlobalStreamId streamId2 = input2.getKey(); DataStream<Tuple> inputStream2 = input2.getValue(); if (mergedInputStream == null) { mergedInputStream = singleInputStream .connect(inputStream2) .flatMap( new TwoFlinkStreamsMerger( streamId1, inputSchema1, streamId2, this.outputStreams .get(streamId2.get_componentId()) .get(streamId2.get_streamId()))) .returns(StormTuple.class); } else { mergedInputStream = mergedInputStream .connect(inputStream2) .flatMap( new StormFlinkStreamMerger( streamId2, this.outputStreams .get(streamId2.get_componentId()) .get(streamId2.get_streamId()))) .returns(StormTuple.class); } } final HashMap<String, Fields> boltOutputs = this.outputStreams.get(boltId); final FlinkOutputFieldsDeclarer declarer = this.declarers.get(boltId); final SingleOutputStreamOperator<?, ?> outputStream; if (boltOutputs.size() < 2) { // single output stream or sink String outputStreamId; if (boltOutputs.size() == 1) { outputStreamId = (String) boltOutputs.keySet().toArray()[0]; } else { outputStreamId = null; } final TypeInformation<Tuple> outType = declarer.getOutputType(outputStreamId); final SingleOutputStreamOperator<Tuple, ?> outStream; // only one input if (inputStreams.entrySet().size() == 1) { BoltWrapper<Tuple, Tuple> boltWrapper = new BoltWrapper<>(bolt, boltId, inputStreamId1, inputComponentId1, inputSchema1, null); boltWrapper.setStormTopology(stormTopology); outStream = singleInputStream.transform(boltId, outType, boltWrapper); } else { MergedInputsBoltWrapper<Tuple, Tuple> boltWrapper = new MergedInputsBoltWrapper<Tuple, Tuple>(bolt, boltId, null); boltWrapper.setStormTopology(stormTopology); outStream = mergedInputStream.transform(boltId, outType, boltWrapper); } if (outType != null) { // only for non-sink nodes final HashMap<String, DataStream<Tuple>> op = new HashMap<>(); op.put(outputStreamId, outStream); availableInputs.put(boltId, op); } outputStream = outStream; } else { final TypeInformation<SplitStreamType<Tuple>> outType = (TypeInformation) TypeExtractor.getForClass(SplitStreamType.class); final SingleOutputStreamOperator<SplitStreamType<Tuple>, ?> multiStream; // only one input if (inputStreams.entrySet().size() == 1) { final BoltWrapper<Tuple, SplitStreamType<Tuple>> boltWrapperMultipleOutputs = new BoltWrapper<>(bolt, boltId, inputStreamId1, inputComponentId1, inputSchema1, null); boltWrapperMultipleOutputs.setStormTopology(stormTopology); multiStream = singleInputStream.transform(boltId, outType, boltWrapperMultipleOutputs); } else { final MergedInputsBoltWrapper<Tuple, SplitStreamType<Tuple>> boltWrapperMultipleOutputs = new MergedInputsBoltWrapper<Tuple, SplitStreamType<Tuple>>(bolt, boltId, null); boltWrapperMultipleOutputs.setStormTopology(stormTopology); multiStream = mergedInputStream.transform(boltId, outType, boltWrapperMultipleOutputs); } final SplitStream<SplitStreamType<Tuple>> splitStream = multiStream.split(new StormStreamSelector<Tuple>()); final HashMap<String, DataStream<Tuple>> op = new HashMap<>(); for (String outputStreamId : boltOutputs.keySet()) { op.put( outputStreamId, splitStream.select(outputStreamId).map(new SplitStreamMapper<Tuple>())); SingleOutputStreamOperator<Tuple, ?> outStream = splitStream.select(outputStreamId).map(new SplitStreamMapper<Tuple>()); outStream.getTransformation().setOutputType(declarer.getOutputType(outputStreamId)); op.put(outputStreamId, outStream); } availableInputs.put(boltId, op); outputStream = multiStream; } return outputStream; }
/** * Tests {@link SingleOutputStreamOperator#name(String)} functionality. * * @throws Exception */ @Test public void testNaming() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStream<Long> dataStream1 = env.generateSequence(0, 0) .name("testSource1") .map( new MapFunction<Long, Long>() { @Override public Long map(Long value) throws Exception { return null; } }) .name("testMap"); DataStream<Long> dataStream2 = env.generateSequence(0, 0) .name("testSource2") .map( new MapFunction<Long, Long>() { @Override public Long map(Long value) throws Exception { return null; } }) .name("testMap"); DataStreamSink<Long> connected = dataStream1 .connect(dataStream2) .flatMap( new CoFlatMapFunction<Long, Long, Long>() { private static final long serialVersionUID = 1L; @Override public void flatMap1(Long value, Collector<Long> out) throws Exception {} @Override public void flatMap2(Long value, Collector<Long> out) throws Exception {} }) .name("testCoFlatMap") .windowAll(GlobalWindows.create()) .trigger(PurgingTrigger.of(CountTrigger.of(10))) .fold( 0L, new FoldFunction<Long, Long>() { private static final long serialVersionUID = 1L; @Override public Long fold(Long accumulator, Long value) throws Exception { return null; } }) .name("testWindowFold") .print(); // test functionality through the operator names in the execution plan String plan = env.getExecutionPlan(); assertTrue(plan.contains("testSource1")); assertTrue(plan.contains("testSource2")); assertTrue(plan.contains("testMap")); assertTrue(plan.contains("testMap")); assertTrue(plan.contains("testCoFlatMap")); assertTrue(plan.contains("testWindowFold")); }
@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()); } }
/** * Tests that {@link DataStream#groupBy} and {@link DataStream#partitionByHash} result in * different and correct topologies. Does the some for the {@link ConnectedDataStream}. */ @Test @SuppressWarnings("unchecked") public void testPartitioning() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); DataStream src1 = env.fromElements(new Tuple2<Long, Long>(0L, 0L)); DataStream src2 = env.fromElements(new Tuple2<Long, Long>(0L, 0L)); ConnectedDataStream connected = src1.connect(src2); // Testing DataStream grouping DataStream group1 = src1.groupBy(0); DataStream group2 = src1.groupBy(1, 0); DataStream group3 = src1.groupBy("f0"); DataStream group4 = src1.groupBy(new FirstSelector()); int id1 = createDownStreamId(group1); int id2 = createDownStreamId(group2); int id3 = createDownStreamId(group3); int id4 = createDownStreamId(group4); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id1))); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id2))); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id3))); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), id4))); assertTrue(isGrouped(group1)); assertTrue(isGrouped(group2)); assertTrue(isGrouped(group3)); assertTrue(isGrouped(group4)); // Testing DataStream partitioning DataStream partition1 = src1.partitionByHash(0); DataStream partition2 = src1.partitionByHash(1, 0); DataStream partition3 = src1.partitionByHash("f0"); DataStream partition4 = src1.partitionByHash(new FirstSelector()); int pid1 = createDownStreamId(partition1); int pid2 = createDownStreamId(partition2); int pid3 = createDownStreamId(partition3); int pid4 = createDownStreamId(partition4); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid1))); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid2))); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid3))); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), pid4))); assertFalse(isGrouped(partition1)); assertFalse(isGrouped(partition3)); assertFalse(isGrouped(partition2)); assertFalse(isGrouped(partition4)); // Testing DataStream custom partitioning Partitioner<Long> longPartitioner = new Partitioner<Long>() { @Override public int partition(Long key, int numPartitions) { return 100; } }; DataStream customPartition1 = src1.partitionCustom(longPartitioner, 0); DataStream customPartition3 = src1.partitionCustom(longPartitioner, "f0"); DataStream customPartition4 = src1.partitionCustom(longPartitioner, new FirstSelector()); int cid1 = createDownStreamId(customPartition1); int cid2 = createDownStreamId(customPartition3); int cid3 = createDownStreamId(customPartition4); assertTrue(isCustomPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), cid1))); assertTrue(isCustomPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), cid2))); assertTrue(isCustomPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), cid3))); assertFalse(isGrouped(customPartition1)); assertFalse(isGrouped(customPartition3)); assertFalse(isGrouped(customPartition4)); // Testing ConnectedDataStream grouping ConnectedDataStream connectedGroup1 = connected.groupBy(0, 0); Integer downStreamId1 = createDownStreamId(connectedGroup1); ConnectedDataStream connectedGroup2 = connected.groupBy(new int[] {0}, new int[] {0}); Integer downStreamId2 = createDownStreamId(connectedGroup2); ConnectedDataStream connectedGroup3 = connected.groupBy("f0", "f0"); Integer downStreamId3 = createDownStreamId(connectedGroup3); ConnectedDataStream connectedGroup4 = connected.groupBy(new String[] {"f0"}, new String[] {"f0"}); Integer downStreamId4 = createDownStreamId(connectedGroup4); ConnectedDataStream connectedGroup5 = connected.groupBy(new FirstSelector(), new FirstSelector()); Integer downStreamId5 = createDownStreamId(connectedGroup5); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId1))); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId1))); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId2))); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId2))); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId3))); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId3))); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId4))); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId4))); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), downStreamId5))); assertTrue(isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), downStreamId5))); assertTrue(isGrouped(connectedGroup1)); assertTrue(isGrouped(connectedGroup2)); assertTrue(isGrouped(connectedGroup3)); assertTrue(isGrouped(connectedGroup4)); assertTrue(isGrouped(connectedGroup5)); // Testing ConnectedDataStream partitioning ConnectedDataStream connectedPartition1 = connected.partitionByHash(0, 0); Integer connectDownStreamId1 = createDownStreamId(connectedPartition1); ConnectedDataStream connectedPartition2 = connected.partitionByHash(new int[] {0}, new int[] {0}); Integer connectDownStreamId2 = createDownStreamId(connectedPartition2); ConnectedDataStream connectedPartition3 = connected.partitionByHash("f0", "f0"); Integer connectDownStreamId3 = createDownStreamId(connectedPartition3); ConnectedDataStream connectedPartition4 = connected.partitionByHash(new String[] {"f0"}, new String[] {"f0"}); Integer connectDownStreamId4 = createDownStreamId(connectedPartition4); ConnectedDataStream connectedPartition5 = connected.partitionByHash(new FirstSelector(), new FirstSelector()); Integer connectDownStreamId5 = createDownStreamId(connectedPartition5); assertTrue( isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), connectDownStreamId1))); assertTrue( isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), connectDownStreamId1))); assertTrue( isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), connectDownStreamId2))); assertTrue( isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), connectDownStreamId2))); assertTrue( isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), connectDownStreamId3))); assertTrue( isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), connectDownStreamId3))); assertTrue( isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), connectDownStreamId4))); assertTrue( isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), connectDownStreamId4))); assertTrue( isPartitioned(env.getStreamGraph().getStreamEdge(src1.getId(), connectDownStreamId5))); assertTrue( isPartitioned(env.getStreamGraph().getStreamEdge(src2.getId(), connectDownStreamId5))); assertFalse(isGrouped(connectedPartition1)); assertFalse(isGrouped(connectedPartition2)); assertFalse(isGrouped(connectedPartition3)); assertFalse(isGrouped(connectedPartition4)); assertFalse(isGrouped(connectedPartition5)); }