/** * Applies an aggregation that gives the maximum value of every window of the data stream at the * given position. * * @param positionToMax The position to maximize * @return The transformed DataStream. */ public SingleOutputStreamOperator<T, ?> max(int positionToMax) { return aggregate( new ComparableAggregator<>( positionToMax, input.getType(), AggregationFunction.AggregationType.MAX, input.getExecutionConfig())); }
/** * Applies an aggregation that that gives the maximum element of the pojo data stream by the given * field expression for every window. A field expression is either the name of a public field or a * getter method with parentheses of the {@link DataStream}S underlying type. A dot can be used to * drill down into objects, as in {@code "field1.getInnerField2()" }. * * @param field The field expression based on which the aggregation will be applied. * @param first If True then in case of field equality the first object will be returned * @return The transformed DataStream. */ public SingleOutputStreamOperator<T, ?> maxBy(String field, boolean first) { return aggregate( new ComparableAggregator<>( field, input.getType(), AggregationFunction.AggregationType.MAXBY, first, input.getExecutionConfig())); }
/** * Applies an aggregation that that gives the maximum value of the pojo data stream at the given * field expression for every window. A field expression is either the name of a public field or a * getter method with parentheses of the {@link DataStream DataStreams} underlying type. A dot can * be used to drill down into objects, as in {@code "field1.getInnerField2()" }. * * @param field The field expression based on which the aggregation will be applied. * @return The transformed DataStream. */ public SingleOutputStreamOperator<T, ?> max(String field) { return aggregate( new ComparableAggregator<>( field, input.getType(), AggregationFunction.AggregationType.MAX, false, input.getExecutionConfig())); }
/** * Applies an aggregation that gives the minimum element of every window of the data stream by the * given position. If more elements have the same minimum value the operator returns either the * first or last one depending on the parameter setting. * * @param positionToMinBy The position to minimize * @param first If true, then the operator return the first element with the minimum value, * otherwise returns the last * @return The transformed DataStream. */ public SingleOutputStreamOperator<T, ?> minBy(int positionToMinBy, boolean first) { return aggregate( new ComparableAggregator<>( positionToMinBy, input.getType(), AggregationFunction.AggregationType.MINBY, first, input.getExecutionConfig())); }
/** * Applies an aggregation that sums every window of the pojo data stream at the given field for * every window. * * <p>A field expression is either the name of a public field or a getter method with parentheses * of the stream's underlying type. A dot can be used to drill down into objects, as in {@code * "field1.getInnerField2()" }. * * @param field The field to sum * @return The transformed DataStream. */ public SingleOutputStreamOperator<T, ?> sum(String field) { return aggregate(new SumAggregator<>(field, input.getType(), input.getExecutionConfig())); }
/** * Applies an aggregation that sums every window of the data stream at the given position. * * @param positionToSum The position in the tuple/array to sum * @return The transformed DataStream. */ public SingleOutputStreamOperator<T, ?> sum(int positionToSum) { return aggregate( new SumAggregator<>(positionToSum, input.getType(), input.getExecutionConfig())); }