/** * Applies a FlatMap transformation on a {@link DataStream}. The transformation calls a {@link * FlatMapFunction} for each element of the DataStream. Each FlatMapFunction call can return any * number of elements including none. The user can also extend {@link RichFlatMapFunction} to gain * access to other features provided by the {@link * org.apache.flink.api.common.functions.RichFunction} interface. * * @param flatMapper The FlatMapFunction that is called for each element of the DataStream * @param <R> output type * @return The transformed {@link DataStream}. */ public <R> SingleOutputStreamOperator<R> flatMap(FlatMapFunction<T, R> flatMapper) { TypeInformation<R> outType = TypeExtractor.getFlatMapReturnTypes( clean(flatMapper), getType(), Utils.getCallLocationName(), true); return transform("Flat Map", outType, new StreamFlatMap<>(clean(flatMapper))); }
/** * Applies the given fold function to each 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. * * @param function The fold function. * @return The data stream that is the result of applying the fold function to the window. */ public <R> SingleOutputStreamOperator<R, ?> fold(R initialValue, FoldFunction<T, R> function) { // clean the closure function = input.getExecutionEnvironment().clean(function); TypeInformation<R> resultType = TypeExtractor.getFoldReturnTypes( function, input.getType(), Utils.getCallLocationName(), true); return apply(new FoldAllWindowFunction<W, T, R>(initialValue, function), resultType); }
/** * Finalizes a CoGroup transformation by applying a {@link * org.apache.flink.api.common.functions.RichCoGroupFunction} to groups of elements with * identical keys.<br> * Each CoGroupFunction call returns an arbitrary number of keys. * * @param function The CoGroupFunction that is called for all groups of elements with * identical keys. * @return An CoGroupOperator that represents the co-grouped result DataSet. * @see org.apache.flink.api.common.functions.RichCoGroupFunction * @see DataSet */ public <R> CoGroupOperator<I1, I2, R> with(CoGroupFunction<I1, I2, R> function) { if (function == null) { throw new NullPointerException("CoGroup function must not be null."); } TypeInformation<R> returnType = TypeExtractor.getCoGroupReturnTypes( function, input1.getType(), input2.getType(), Utils.getCallLocationName(), true); return new CoGroupOperator<>( input1, input2, keys1, keys2, input1.clean(function), returnType, groupSortKeyOrderFirst, groupSortKeyOrderSecond, customPartitioner, Utils.getCallLocationName()); }
/** * 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 a CoMap transformation on a {@link ConnectedStreams} and maps the output to a common * type. The transformation calls a {@link CoMapFunction#map1} for each element of the first input * and {@link CoMapFunction#map2} for each element of the second input. Each CoMapFunction call * returns exactly one element. * * @param coMapper The CoMapFunction used to jointly transform the two input DataStreams * @return The transformed {@link DataStream} */ public <R> SingleOutputStreamOperator<R> map(CoMapFunction<IN1, IN2, R> coMapper) { TypeInformation<R> outTypeInfo = TypeExtractor.getBinaryOperatorReturnType( coMapper, CoMapFunction.class, false, true, getType1(), getType2(), Utils.getCallLocationName(), true); return transform("Co-Map", outTypeInfo, new CoStreamMap<>(inputStream1.clean(coMapper))); }
/** * 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); }