Class AllWindowedStream<T,W extends Window>

java.lang.Object
org.apache.flink.streaming.api.datastream.AllWindowedStream<T,W>
Type Parameters:
T - The type of elements in the stream.
W - The type of Window that the WindowAssigner assigns the elements to.

@Public public class AllWindowedStream<T,W extends Window> extends Object
A AllWindowedStream represents a data stream where the stream of elements is split into windows based on a WindowAssigner. Window emission is triggered based on a Trigger.

If an Evictor is specified it will be used to evict elements from the window after evaluation was triggered by the Trigger but before the actual evaluation of the window. When using an evictor, window performance will degrade significantly, since pre-aggregation of window results cannot be used.

Note that the AllWindowedStream is purely an API construct, during runtime the AllWindowedStream will be collapsed together with the operation over the window into one single operation.

  • Constructor Details

  • Method Details

    • trigger

      @PublicEvolving public AllWindowedStream<T,W> trigger(Trigger<? super T,? super W> trigger)
      Sets the Trigger that should be used to trigger window emission.
    • allowedLateness

      @PublicEvolving public AllWindowedStream<T,W> allowedLateness(Duration lateness)
      Sets the time by which elements are allowed to be late. Elements that arrive behind the watermark by more than the specified time will be dropped. By default, the allowed lateness is 0L.

      Setting an allowed lateness is only valid for event-time windows.

    • sideOutputLateData

      @PublicEvolving public AllWindowedStream<T,W> sideOutputLateData(org.apache.flink.util.OutputTag<T> outputTag)
      Send late arriving data to the side output identified by the given OutputTag. Data is considered late after the watermark has passed the end of the window plus the allowed lateness set using allowedLateness(Duration).

      You can get the stream of late data using SingleOutputStreamOperator.getSideOutput(OutputTag) on the SingleOutputStreamOperator resulting from the windowed operation with the same OutputTag.

    • evictor

      @PublicEvolving public AllWindowedStream<T,W> evictor(Evictor<? super T,? super W> evictor)
      Sets the Evictor that should be used to evict elements from a window before emission.

      Note: When using an evictor window performance will degrade significantly, since incremental aggregation of window results cannot be used.

    • reduce

      public SingleOutputStreamOperator<T> reduce(org.apache.flink.api.common.functions.ReduceFunction<T> function)
      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.

      This window will try and incrementally aggregate data as much as the window policies permit. For example, tumbling time windows can aggregate the data, meaning that only one element per key is stored. Sliding time windows will 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 incrementally aggregate, or may need to store extra values in an aggregation tree.

      Parameters:
      function - The reduce function.
      Returns:
      The data stream that is the result of applying the reduce function to the window.
    • reduce

      @PublicEvolving public <R> SingleOutputStreamOperator<R> reduce(org.apache.flink.api.common.functions.ReduceFunction<T> reduceFunction, AllWindowFunction<T,R,W> function)
      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.

      Arriving data is incrementally aggregated using the given reducer.

      Parameters:
      reduceFunction - The reduce function that is used for incremental aggregation.
      function - The window function.
      Returns:
      The data stream that is the result of applying the window function to the window.
    • reduce

      @PublicEvolving public <R> SingleOutputStreamOperator<R> reduce(org.apache.flink.api.common.functions.ReduceFunction<T> reduceFunction, AllWindowFunction<T,R,W> function, org.apache.flink.api.common.typeinfo.TypeInformation<R> resultType)
      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.

      Arriving data is incrementally aggregated using the given reducer.

      Parameters:
      reduceFunction - The reduce function that is used for incremental aggregation.
      function - The window function.
      resultType - Type information for the result type of the window function
      Returns:
      The data stream that is the result of applying the window function to the window.
    • reduce

      @PublicEvolving public <R> SingleOutputStreamOperator<R> reduce(org.apache.flink.api.common.functions.ReduceFunction<T> reduceFunction, ProcessAllWindowFunction<T,R,W> function)
      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.

      Arriving data is incrementally aggregated using the given reducer.

      Parameters:
      reduceFunction - The reduce function that is used for incremental aggregation.
      function - The process window function.
      Returns:
      The data stream that is the result of applying the window function to the window.
    • reduce

      @PublicEvolving public <R> SingleOutputStreamOperator<R> reduce(org.apache.flink.api.common.functions.ReduceFunction<T> reduceFunction, ProcessAllWindowFunction<T,R,W> function, org.apache.flink.api.common.typeinfo.TypeInformation<R> resultType)
      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.

      Arriving data is incrementally aggregated using the given reducer.

      Parameters:
      reduceFunction - The reduce function that is used for incremental aggregation.
      function - The process window function.
      resultType - Type information for the result type of the window function
      Returns:
      The data stream that is the result of applying the window function to the window.
    • aggregate

      @PublicEvolving public <ACC, R> SingleOutputStreamOperator<R> aggregate(org.apache.flink.api.common.functions.AggregateFunction<T,ACC,R> function)
      Applies the given AggregateFunction to each window. The AggregateFunction aggregates all elements of a window into a single result element. The stream of these result elements (one per window) is interpreted as a regular non-windowed stream.
      Type Parameters:
      ACC - The type of the AggregateFunction's accumulator
      R - The type of the elements in the resulting stream, equal to the AggregateFunction's result type
      Parameters:
      function - The aggregation function.
      Returns:
      The data stream that is the result of applying the fold function to the window.
    • aggregate

      @PublicEvolving public <ACC, R> SingleOutputStreamOperator<R> aggregate(org.apache.flink.api.common.functions.AggregateFunction<T,ACC,R> function, org.apache.flink.api.common.typeinfo.TypeInformation<ACC> accumulatorType, org.apache.flink.api.common.typeinfo.TypeInformation<R> resultType)
      Applies the given AggregateFunction to each window. The AggregateFunction aggregates all elements of a window into a single result element. The stream of these result elements (one per window) is interpreted as a regular non-windowed stream.
      Type Parameters:
      ACC - The type of the AggregateFunction's accumulator
      R - The type of the elements in the resulting stream, equal to the AggregateFunction's result type
      Parameters:
      function - The aggregation function.
      Returns:
      The data stream that is the result of applying the aggregation function to the window.
    • aggregate

      @PublicEvolving public <ACC, V, R> SingleOutputStreamOperator<R> aggregate(org.apache.flink.api.common.functions.AggregateFunction<T,ACC,V> aggFunction, AllWindowFunction<V,R,W> windowFunction)
      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.

      Arriving data is incrementally aggregated using the given aggregate function. This means that the window function typically has only a single value to process when called.

      Type Parameters:
      ACC - The type of the AggregateFunction's accumulator
      V - The type of AggregateFunction's result, and the WindowFunction's input
      R - The type of the elements in the resulting stream, equal to the WindowFunction's result type
      Parameters:
      aggFunction - The aggregate function that is used for incremental aggregation.
      windowFunction - The window function.
      Returns:
      The data stream that is the result of applying the window function to the window.
    • aggregate

      @PublicEvolving public <ACC, V, R> SingleOutputStreamOperator<R> aggregate(org.apache.flink.api.common.functions.AggregateFunction<T,ACC,V> aggregateFunction, AllWindowFunction<V,R,W> windowFunction, org.apache.flink.api.common.typeinfo.TypeInformation<ACC> accumulatorType, org.apache.flink.api.common.typeinfo.TypeInformation<R> resultType)
      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.

      Arriving data is incrementally aggregated using the given aggregate function. This means that the window function typically has only a single value to process when called.

      Type Parameters:
      ACC - The type of the AggregateFunction's accumulator
      V - The type of AggregateFunction's result, and the WindowFunction's input
      R - The type of the elements in the resulting stream, equal to the WindowFunction's result type
      Parameters:
      aggregateFunction - The aggregation function that is used for incremental aggregation.
      windowFunction - The window function.
      accumulatorType - Type information for the internal accumulator type of the aggregation function
      resultType - Type information for the result type of the window function
      Returns:
      The data stream that is the result of applying the window function to the window.
    • aggregate

      @PublicEvolving public <ACC, V, R> SingleOutputStreamOperator<R> aggregate(org.apache.flink.api.common.functions.AggregateFunction<T,ACC,V> aggFunction, ProcessAllWindowFunction<V,R,W> windowFunction)
      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.

      Arriving data is incrementally aggregated using the given aggregate function. This means that the window function typically has only a single value to process when called.

      Type Parameters:
      ACC - The type of the AggregateFunction's accumulator
      V - The type of AggregateFunction's result, and the WindowFunction's input
      R - The type of the elements in the resulting stream, equal to the WindowFunction's result type
      Parameters:
      aggFunction - The aggregate function that is used for incremental aggregation.
      windowFunction - The process window function.
      Returns:
      The data stream that is the result of applying the window function to the window.
    • aggregate

      @PublicEvolving public <ACC, V, R> SingleOutputStreamOperator<R> aggregate(org.apache.flink.api.common.functions.AggregateFunction<T,ACC,V> aggregateFunction, ProcessAllWindowFunction<V,R,W> windowFunction, org.apache.flink.api.common.typeinfo.TypeInformation<ACC> accumulatorType, org.apache.flink.api.common.typeinfo.TypeInformation<V> aggregateResultType, org.apache.flink.api.common.typeinfo.TypeInformation<R> resultType)
      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.

      Arriving data is incrementally aggregated using the given aggregate function. This means that the window function typically has only a single value to process when called.

      Type Parameters:
      ACC - The type of the AggregateFunction's accumulator
      V - The type of AggregateFunction's result, and the WindowFunction's input
      R - The type of the elements in the resulting stream, equal to the WindowFunction's result type
      Parameters:
      aggregateFunction - The aggregation function that is used for incremental aggregation.
      windowFunction - The process window function.
      accumulatorType - Type information for the internal accumulator type of the aggregation function
      resultType - Type information for the result type of the window function
      Returns:
      The data stream that is the result of applying the window function to the window.
    • apply

      public <R> SingleOutputStreamOperator<R> apply(AllWindowFunction<T,R,W> function)
      Applies the given window function to each window. The window function is called for each evaluation of the window. The output of the window function is interpreted as a regular non-windowed stream.

      Note that this function requires that all data in the windows is buffered until the window is evaluated, as the function provides no means of incremental aggregation.

      Parameters:
      function - The window function.
      Returns:
      The data stream that is the result of applying the window function to the window.
    • apply

      public <R> SingleOutputStreamOperator<R> apply(AllWindowFunction<T,R,W> function, org.apache.flink.api.common.typeinfo.TypeInformation<R> resultType)
      Applies the given window function to each window. The window function is called for each evaluation of the window. The output of the window function is interpreted as a regular non-windowed stream.

      Note that this function requires that all data in the windows is buffered until the window is evaluated, as the function provides no means of incremental aggregation.

      Parameters:
      function - The window function.
      Returns:
      The data stream that is the result of applying the window function to the window.
    • process

      @PublicEvolving public <R> SingleOutputStreamOperator<R> process(ProcessAllWindowFunction<T,R,W> function)
      Applies the given window function to each window. The window function is called for each evaluation of the window. The output of the window function is interpreted as a regular non-windowed stream.

      Note that this function requires that all data in the windows is buffered until the window is evaluated, as the function provides no means of incremental aggregation.

      Parameters:
      function - The process window function.
      Returns:
      The data stream that is the result of applying the window function to the window.
    • process

      @PublicEvolving public <R> SingleOutputStreamOperator<R> process(ProcessAllWindowFunction<T,R,W> function, org.apache.flink.api.common.typeinfo.TypeInformation<R> resultType)
      Applies the given window function to each window. The window function is called for each evaluation of the window. The output of the window function is interpreted as a regular non-windowed stream.

      Note that this function requires that all data in the windows is buffered until the window is evaluated, as the function provides no means of incremental aggregation.

      Parameters:
      function - The process window function.
      Returns:
      The data stream that is the result of applying the window function to the window.
    • sum

      public SingleOutputStreamOperator<T> sum(int positionToSum)
      Applies an aggregation that sums every window of the data stream at the given position.
      Parameters:
      positionToSum - The position in the tuple/array to sum
      Returns:
      The transformed DataStream.
    • sum

      public SingleOutputStreamOperator<T> sum(String field)
      Applies an aggregation that sums every window of the pojo data stream at the given field for every window.

      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 "field1.getInnerField2()" .

      Parameters:
      field - The field to sum
      Returns:
      The transformed DataStream.
    • min

      public SingleOutputStreamOperator<T> min(int positionToMin)
      Applies an aggregation that gives the minimum value of every window of the data stream at the given position.
      Parameters:
      positionToMin - The position to minimize
      Returns:
      The transformed DataStream.
    • min

      public SingleOutputStreamOperator<T> min(String field)
      Applies an aggregation that gives the minimum 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 DataStreamS underlying type. A dot can be used to drill down into objects, as in "field1.getInnerField2()" .

      Parameters:
      field - The field expression based on which the aggregation will be applied.
      Returns:
      The transformed DataStream.
    • minBy

      public SingleOutputStreamOperator<T> minBy(int positionToMinBy)
      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 the first element by default.
      Parameters:
      positionToMinBy - The position to minimize by
      Returns:
      The transformed DataStream.
    • minBy

      public SingleOutputStreamOperator<T> minBy(String positionToMinBy)
      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 the first element by default.
      Parameters:
      positionToMinBy - The position to minimize by
      Returns:
      The transformed DataStream.
    • minBy

      public SingleOutputStreamOperator<T> minBy(int positionToMinBy, boolean first)
      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.
      Parameters:
      positionToMinBy - The position to minimize
      first - If true, then the operator return the first element with the minimum value, otherwise returns the last
      Returns:
      The transformed DataStream.
    • minBy

      public SingleOutputStreamOperator<T> minBy(String field, boolean first)
      Applies an aggregation that gives the minimum 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 DataStreams underlying type. A dot can be used to drill down into objects, as in "field1.getInnerField2()" .
      Parameters:
      field - The field expression based on which the aggregation will be applied.
      first - If True then in case of field equality the first object will be returned
      Returns:
      The transformed DataStream.
    • max

      public SingleOutputStreamOperator<T> max(int positionToMax)
      Applies an aggregation that gives the maximum value of every window of the data stream at the given position.
      Parameters:
      positionToMax - The position to maximize
      Returns:
      The transformed DataStream.
    • max

      public SingleOutputStreamOperator<T> max(String field)
      Applies an aggregation 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 DataStreams underlying type. A dot can be used to drill down into objects, as in "field1.getInnerField2()" .
      Parameters:
      field - The field expression based on which the aggregation will be applied.
      Returns:
      The transformed DataStream.
    • maxBy

      public SingleOutputStreamOperator<T> maxBy(int positionToMaxBy)
      Applies an aggregation that gives the maximum element of every window of the data stream by the given position. If more elements have the same maximum value the operator returns the first by default.
      Parameters:
      positionToMaxBy - The position to maximize by
      Returns:
      The transformed DataStream.
    • maxBy

      public SingleOutputStreamOperator<T> maxBy(String positionToMaxBy)
      Applies an aggregation that gives the maximum element of every window of the data stream by the given position. If more elements have the same maximum value the operator returns the first by default.
      Parameters:
      positionToMaxBy - The position to maximize by
      Returns:
      The transformed DataStream.
    • maxBy

      public SingleOutputStreamOperator<T> maxBy(int positionToMaxBy, boolean first)
      Applies an aggregation that gives the maximum element of every window of the data stream by the given position. If more elements have the same maximum value the operator returns either the first or last one depending on the parameter setting.
      Parameters:
      positionToMaxBy - The position to maximize by
      first - If true, then the operator return the first element with the maximum value, otherwise returns the last
      Returns:
      The transformed DataStream.
    • maxBy

      public SingleOutputStreamOperator<T> maxBy(String field, boolean first)
      Applies an aggregation 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 DataStreamS underlying type. A dot can be used to drill down into objects, as in "field1.getInnerField2()" .
      Parameters:
      field - The field expression based on which the aggregation will be applied.
      first - If True then in case of field equality the first object will be returned
      Returns:
      The transformed DataStream.
    • getExecutionEnvironment

      public StreamExecutionEnvironment getExecutionEnvironment()
    • getInputType

      public org.apache.flink.api.common.typeinfo.TypeInformation<T> getInputType()