public class AllWindowedStream<T,W extends Window> extends Object
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 and API construct, during runtime
the AllWindowedStream()
will be collapsed together with the
operation over the window into one single operation.
Constructor and Description |
---|
AllWindowedStream(AllWindowedStream<T,W> javaStream) |
Modifier and Type | Method and Description |
---|---|
DataStream<T> |
aggregate(AggregationFunction.AggregationType aggregationType,
int position) |
<R> DataStream<R> |
apply(AllWindowFunction<T,R,W> function,
TypeInformation<R> evidence$3)
Applies the given window function to each window.
|
<R> DataStream<R> |
apply(scala.Function2<T,T,T> preAggregator,
scala.Function3<W,scala.collection.Iterable<T>,Collector<R>,scala.runtime.BoxedUnit> windowFunction,
TypeInformation<R> evidence$6)
Applies the given window function to each window.
|
<R> DataStream<R> |
apply(scala.Function3<W,scala.collection.Iterable<T>,Collector<R>,scala.runtime.BoxedUnit> function,
TypeInformation<R> evidence$4)
Applies the given window function to each window.
|
<R> DataStream<R> |
apply(ReduceFunction<T> preAggregator,
AllWindowFunction<T,R,W> windowFunction,
TypeInformation<R> evidence$5)
Applies the given window function to each window.
|
<R> DataStream<R> |
apply(R initialValue,
FoldFunction<T,R> preAggregator,
AllWindowFunction<R,R,W> windowFunction,
TypeInformation<R> evidence$7)
Applies the given window function to each window.
|
<R> DataStream<R> |
apply(R initialValue,
scala.Function2<R,T,R> preAggregator,
scala.Function3<W,scala.collection.Iterable<R>,Collector<R>,scala.runtime.BoxedUnit> windowFunction,
TypeInformation<R> evidence$8)
Applies the given window function to each window.
|
<F> F |
clean(F f)
Returns a "closure-cleaned" version of the given function.
|
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. |
<R> DataStream<R> |
fold(R initialValue,
FoldFunction<T,R> function,
TypeInformation<R> evidence$1)
Applies the given fold function to each window.
|
<R> DataStream<R> |
fold(R initialValue,
scala.Function2<R,T,R> function,
TypeInformation<R> evidence$2)
Applies the given fold function to each window.
|
DataStream<T> |
max(int position)
Applies an aggregation that that gives the maximum of the elements in the window at
the given position.
|
DataStream<T> |
max(String field)
Applies an aggregation that that gives the maximum of the elements in the window at
the given field.
|
DataStream<T> |
maxBy(int position)
Applies an aggregation that that gives the maximum element of the window by
the given position.
|
DataStream<T> |
maxBy(String field)
Applies an aggregation that that gives the maximum element of the window by
the given field.
|
DataStream<T> |
min(int position)
Applies an aggregation that that gives the minimum of the elements in the window at
the given position.
|
DataStream<T> |
min(String field)
Applies an aggregation that that gives the minimum of the elements in the window at
the given field.
|
DataStream<T> |
minBy(int position)
Applies an aggregation that that gives the minimum element of the window by
the given position.
|
DataStream<T> |
minBy(String field)
Applies an aggregation that that gives the minimum element of the window by
the given field.
|
DataStream<T> |
reduce(scala.Function2<T,T,T> function)
Applies a reduce function to the window.
|
DataStream<T> |
reduce(ReduceFunction<T> function)
Applies a reduce function to the window.
|
DataStream<T> |
sum(int position)
Applies an aggregation that sums the elements in the window at the given position.
|
DataStream<T> |
sum(String field)
Applies an aggregation that sums the elements in the window at the given field.
|
AllWindowedStream<T,W> |
trigger(Trigger<? super T,? super W> trigger)
Sets the
Trigger that should be used to trigger window emission. |
public AllWindowedStream(AllWindowedStream<T,W> javaStream)
public AllWindowedStream<T,W> trigger(Trigger<? super T,? super W> trigger)
Trigger
that should be used to trigger window emission.public AllWindowedStream<T,W> evictor(Evictor<? super T,? super W> evictor)
Evictor
that should be used to evict elements from a window before emission.
Note: When using an evictor window performance will degrade significantly, since pre-aggregation of window results cannot be used.
public DataStream<T> reduce(ReduceFunction<T> function)
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.
function
- The reduce function.public DataStream<T> reduce(scala.Function2<T,T,T> function)
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.
function
- The reduce function.public <R> DataStream<R> fold(R initialValue, FoldFunction<T,R> function, TypeInformation<R> evidence$1)
function
- The fold function.public <R> DataStream<R> fold(R initialValue, scala.Function2<R,T,R> function, TypeInformation<R> evidence$2)
function
- The fold function.public <R> DataStream<R> apply(AllWindowFunction<T,R,W> function, TypeInformation<R> evidence$3)
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.
function
- The window function.public <R> DataStream<R> apply(scala.Function3<W,scala.collection.Iterable<T>,Collector<R>,scala.runtime.BoxedUnit> function, TypeInformation<R> evidence$4)
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.
function
- The window function.public <R> DataStream<R> apply(ReduceFunction<T> preAggregator, AllWindowFunction<T,R,W> windowFunction, TypeInformation<R> evidence$5)
Arriving data is pre-aggregated using the given pre-aggregation reducer.
preAggregator
- The reduce function that is used for pre-aggregationwindowFunction
- The window function.public <R> DataStream<R> apply(scala.Function2<T,T,T> preAggregator, scala.Function3<W,scala.collection.Iterable<T>,Collector<R>,scala.runtime.BoxedUnit> windowFunction, TypeInformation<R> evidence$6)
Arriving data is pre-aggregated using the given pre-aggregation reducer.
preAggregator
- The reduce function that is used for pre-aggregationwindowFunction
- The window function.public <R> DataStream<R> apply(R initialValue, FoldFunction<T,R> preAggregator, AllWindowFunction<R,R,W> windowFunction, TypeInformation<R> evidence$7)
Arriving data is pre-aggregated using the given pre-aggregation folder.
initialValue
- Initial value of the foldpreAggregator
- The reduce function that is used for pre-aggregationwindowFunction
- The window function.public <R> DataStream<R> apply(R initialValue, scala.Function2<R,T,R> preAggregator, scala.Function3<W,scala.collection.Iterable<R>,Collector<R>,scala.runtime.BoxedUnit> windowFunction, TypeInformation<R> evidence$8)
Arriving data is pre-aggregated using the given pre-aggregation folder.
initialValue
- Initial value of the foldpreAggregator
- The reduce function that is used for pre-aggregationwindowFunction
- The window function.public DataStream<T> max(int position)
public DataStream<T> max(String field)
public DataStream<T> min(int position)
public DataStream<T> min(String field)
public DataStream<T> sum(int position)
public DataStream<T> sum(String field)
public DataStream<T> maxBy(int position)
public DataStream<T> maxBy(String field)
public DataStream<T> minBy(int position)
public DataStream<T> minBy(String field)
public DataStream<T> aggregate(AggregationFunction.AggregationType aggregationType, int position)
public <F> F clean(F f)
ExecutionConfig
.Copyright © 2014–2017 The Apache Software Foundation. All rights reserved.