T - The type of the elements in this stream.@PublicEvolving
public interface PartitionWindowedStream<T>
PartitionWindowedStream represents a data stream that collects all records of each
partition separately into a full window. Window emission will be triggered at the end of inputs.
For non-keyed DataStream, a partition contains all records of a subtask. For KeyedStream, a partition contains all records of a key.| Modifier and Type | Method and Description |
|---|---|
<ACC,R> SingleOutputStreamOperator<R> |
aggregate(org.apache.flink.api.common.functions.AggregateFunction<T,ACC,R> aggregateFunction)
Applies an aggregate transformation on the records of the window.
|
<R> SingleOutputStreamOperator<R> |
mapPartition(org.apache.flink.api.common.functions.MapPartitionFunction<T,R> mapPartitionFunction)
Process the records of the window by
MapPartitionFunction. |
SingleOutputStreamOperator<T> |
reduce(org.apache.flink.api.common.functions.ReduceFunction<T> reduceFunction)
Applies a reduce transformation on the records of the window.
|
SingleOutputStreamOperator<T> |
sortPartition(int field,
org.apache.flink.api.common.operators.Order order)
Sorts the records of the window on the specified field in the specified order.
|
<K> SingleOutputStreamOperator<T> |
sortPartition(org.apache.flink.api.java.functions.KeySelector<T,K> keySelector,
org.apache.flink.api.common.operators.Order order)
Sorts the records according to a
KeySelector in the specified order. |
SingleOutputStreamOperator<T> |
sortPartition(String field,
org.apache.flink.api.common.operators.Order order)
Sorts the records of the window on the specified field in the specified order.
|
<R> SingleOutputStreamOperator<R> mapPartition(org.apache.flink.api.common.functions.MapPartitionFunction<T,R> mapPartitionFunction)
MapPartitionFunction.R - The type of map partition result.mapPartitionFunction - The map partition function.SingleOutputStreamOperator<T> reduce(org.apache.flink.api.common.functions.ReduceFunction<T> reduceFunction)
reduceFunction - The reduce function.<ACC,R> SingleOutputStreamOperator<R> aggregate(org.apache.flink.api.common.functions.AggregateFunction<T,ACC,R> aggregateFunction)
ACC - The type of accumulator in aggregate function.R - The type of aggregate function result.aggregateFunction - The aggregate function.SingleOutputStreamOperator<T> sortPartition(int field, org.apache.flink.api.common.operators.Order order)
Tuple.
This operator will use managed memory for the sort.For NonKeyedPartitionWindowedStream, the managed memory size can be set by ExecutionOptions.SORT_PARTITION_MEMORY. For KeyedPartitionWindowedStream, the
managed memory size can be set by ExecutionOptions.SORT_KEYED_PARTITION_MEMORY.
field - The field 1-based index on which records is sorted.order - The order in which records is sorted.SingleOutputStreamOperator<T> sortPartition(String field, org.apache.flink.api.common.operators.Order order)
PojoTypeInfo. A type is considered a Flink POJO type, if
it fulfills the conditions below.
This operator will use managed memory for the sort.For NonKeyedPartitionWindowedStream, the managed memory size can be set by ExecutionOptions.SORT_PARTITION_MEMORY. For KeyedPartitionWindowedStream, the
managed memory size can be set by ExecutionOptions.SORT_KEYED_PARTITION_MEMORY.
field - The field expression referring to the field on which records is sorted.order - The order in which records is sorted.<K> SingleOutputStreamOperator<T> sortPartition(org.apache.flink.api.java.functions.KeySelector<T,K> keySelector, org.apache.flink.api.common.operators.Order order)
KeySelector in the specified order.
This operator will use managed memory for the sort.For NonKeyedPartitionWindowedStream, the managed memory size can be set by ExecutionOptions.SORT_PARTITION_MEMORY. For KeyedPartitionWindowedStream, the
managed memory size can be set by ExecutionOptions.SORT_KEYED_PARTITION_MEMORY.
keySelector - The key selector to extract key from the records for sorting.order - The order in which records is sorted.Copyright © 2014–2025 The Apache Software Foundation. All rights reserved.