public static void main(final String[] args) throws Exception { if (!parseParameters(args)) { return; } // set up the execution environment final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); // get input data final DataStream<String> text = getTextDataStream(env); final DataStream<Tuple2<String, Integer>> counts = text // split up the lines in pairs (2-tuples) containing: (word,1) // this is done by a bolt that is wrapped accordingly .transform( "BoltTokenizer", TypeExtractor.getForObject(new Tuple2<String, Integer>("", 0)), new BoltWrapper<String, Tuple2<String, Integer>>(new BoltTokenizer())) // group by the tuple field "0" and sum up tuple field "1" .keyBy(0) .sum(1); // emit result if (fileOutput) { counts.writeAsText(outputPath); } else { counts.print(); } // execute program env.execute("Streaming WordCount with bolt tokenizer"); }
/** * 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 testOneInputOperatorWithoutChaining() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setParallelism(1); DataStream<String> source = env.addSource(new InfiniteTestSource()); source.transform( "Custom Operator", BasicTypeInfo.STRING_TYPE_INFO, new TimerOperator(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); }
/** * Applies a reduce function to the window. The window function is called for each evaluation of * the window for each key individually. The output of the reduce function is interpreted as a * regular non-windowed stream. * * <p>This window will try and pre-aggregate data as much as the window policies permit. For * example, tumbling time windows can perfectly pre-aggregate the data, meaning that only one * element per key is stored. Sliding time windows will pre-aggregate on the granularity of the * slide interval, so a few elements are stored per key (one per slide interval). Custom windows * may not be able to pre-aggregate, or may need to store extra values in an aggregation tree. * * @param function The reduce function. * @return The data stream that is the result of applying the reduce function to the window. */ public SingleOutputStreamOperator<T, ?> reduce(ReduceFunction<T> function) { if (function instanceof RichFunction) { throw new UnsupportedOperationException( "ReduceFunction of reduce can not be a RichFunction. " + "Please use apply(ReduceFunction, WindowFunction) instead."); } // clean the closure function = input.getExecutionEnvironment().clean(function); String callLocation = Utils.getCallLocationName(); String udfName = "Reduce at " + callLocation; SingleOutputStreamOperator<T, ?> result = createFastTimeOperatorIfValid(function, input.getType(), udfName); if (result != null) { return result; } String opName = "NonParallelTriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")"; OneInputStreamOperator<T, T> operator; boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime; if (evictor != null) { operator = new EvictingNonKeyedWindowOperator<>( windowAssigner, windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()), new HeapWindowBuffer.Factory<T>(), new ReduceIterableAllWindowFunction<W, T>(function), trigger, evictor) .enableSetProcessingTime(setProcessingTime); } else { operator = new NonKeyedWindowOperator<>( windowAssigner, windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()), new PreAggregatingHeapWindowBuffer.Factory<>(function), new ReduceIterableAllWindowFunction<W, T>(function), trigger) .enableSetProcessingTime(setProcessingTime); } return input.transform(opName, input.getType(), operator).setParallelism(1); }
/** * Applies the given window function to each window. The window function is called for each * evaluation of the window for each key individually. The output of the window function is * interpreted as a regular non-windowed stream. * * <p>Arriving data is pre-aggregated using the given pre-aggregation reducer. * * @param preAggregator The reduce function that is used for pre-aggregation * @param function The window function. * @param resultType Type information for the result type of the window function * @return The data stream that is the result of applying the window function to the window. */ public <R> SingleOutputStreamOperator<R, ?> apply( ReduceFunction<T> preAggregator, AllWindowFunction<T, R, W> function, TypeInformation<R> resultType) { if (preAggregator instanceof RichFunction) { throw new UnsupportedOperationException("Pre-aggregator of apply can not be a RichFunction."); } // clean the closures function = input.getExecutionEnvironment().clean(function); preAggregator = input.getExecutionEnvironment().clean(preAggregator); String callLocation = Utils.getCallLocationName(); String udfName = "WindowApply at " + callLocation; String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")"; OneInputStreamOperator<T, R> operator; boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime; if (evictor != null) { operator = new EvictingNonKeyedWindowOperator<>( windowAssigner, windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()), new HeapWindowBuffer.Factory<T>(), new ReduceApplyAllWindowFunction<>(preAggregator, function), trigger, evictor) .enableSetProcessingTime(setProcessingTime); } else { operator = new NonKeyedWindowOperator<>( windowAssigner, windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()), new PreAggregatingHeapWindowBuffer.Factory<>(preAggregator), new ReduceApplyAllWindowFunction<>(preAggregator, function), trigger) .enableSetProcessingTime(setProcessingTime); } return input.transform(opName, resultType, operator).setParallelism(1); }
/** * Applies the given window function to each window. The window function is called for each * evaluation of the window for each key individually. The output of the window function is * interpreted as a regular non-windowed stream. * * <p>Not that this function requires that all data in the windows is buffered until the window is * evaluated, as the function provides no means of pre-aggregation. * * @param function The window function. * @return The data stream that is the result of applying the window function to the window. */ public <R> SingleOutputStreamOperator<R, ?> apply( AllWindowFunction<Iterable<T>, R, W> function, TypeInformation<R> resultType) { // clean the closure function = input.getExecutionEnvironment().clean(function); String callLocation = Utils.getCallLocationName(); String udfName = "WindowApply at " + callLocation; SingleOutputStreamOperator<R, ?> result = createFastTimeOperatorIfValid(function, resultType, udfName); if (result != null) { return result; } String opName = "TriggerWindow(" + windowAssigner + ", " + trigger + ", " + udfName + ")"; NonKeyedWindowOperator<T, R, W> operator; boolean setProcessingTime = input.getExecutionEnvironment().getStreamTimeCharacteristic() == TimeCharacteristic.ProcessingTime; if (evictor != null) { operator = new EvictingNonKeyedWindowOperator<>( windowAssigner, windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()), new HeapWindowBuffer.Factory<T>(), function, trigger, evictor) .enableSetProcessingTime(setProcessingTime); } else { operator = new NonKeyedWindowOperator<>( windowAssigner, windowAssigner.getWindowSerializer(getExecutionEnvironment().getConfig()), new HeapWindowBuffer.Factory<T>(), function, trigger) .enableSetProcessingTime(setProcessingTime); } return input.transform(opName, resultType, operator).setParallelism(1); }
@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; }