@PublicEvolving public class ProcessingTimeSessionWindows extends MergingWindowAssigner<Object,TimeWindow>
WindowAssigner that windows elements into sessions based on the current processing
time. Windows cannot overlap.
For example, in order to window into windows of 1 minute, every 10 seconds:
DataStream<Tuple2<String, Integer>> in = ...;
KeyedStream<String, Tuple2<String, Integer>> keyed = in.keyBy(...);
WindowedStream<Tuple2<String, Integer>, String, TimeWindows> windowed =
keyed.window(ProcessingTimeSessionWindows.withGap(Duration.ofMinutes(1)));
MergingWindowAssigner.MergeCallback<W>WindowAssigner.WindowAssignerContext| Modifier and Type | Field and Description |
|---|---|
protected long |
sessionTimeout |
| Modifier | Constructor and Description |
|---|---|
protected |
ProcessingTimeSessionWindows(long sessionTimeout) |
| Modifier and Type | Method and Description |
|---|---|
Collection<TimeWindow> |
assignWindows(Object element,
long timestamp,
WindowAssigner.WindowAssignerContext context)
Returns a
Collection of windows that should be assigned to the element. |
Trigger<Object,TimeWindow> |
getDefaultTrigger()
Returns the default trigger associated with this
WindowAssigner. |
Trigger<Object,TimeWindow> |
getDefaultTrigger(StreamExecutionEnvironment env)
Returns the default trigger associated with this
WindowAssigner. |
org.apache.flink.api.common.typeutils.TypeSerializer<TimeWindow> |
getWindowSerializer(org.apache.flink.api.common.ExecutionConfig executionConfig)
Returns a
TypeSerializer for serializing windows that are assigned by this WindowAssigner. |
boolean |
isEventTime()
Returns
true if elements are assigned to windows based on event time, false
otherwise. |
void |
mergeWindows(Collection<TimeWindow> windows,
MergingWindowAssigner.MergeCallback<TimeWindow> c)
Merge overlapping
TimeWindows. |
String |
toString() |
static <T> DynamicProcessingTimeSessionWindows<T> |
withDynamicGap(SessionWindowTimeGapExtractor<T> sessionWindowTimeGapExtractor)
Creates a new
SessionWindows WindowAssigner that assigns elements to sessions
based on the element timestamp. |
static ProcessingTimeSessionWindows |
withGap(Duration size)
Creates a new
SessionWindows WindowAssigner that assigns elements to sessions
based on the element timestamp. |
static ProcessingTimeSessionWindows |
withGap(Time size)
Deprecated.
|
protected ProcessingTimeSessionWindows(long sessionTimeout)
public Collection<TimeWindow> assignWindows(Object element, long timestamp, WindowAssigner.WindowAssignerContext context)
WindowAssignerCollection of windows that should be assigned to the element.assignWindows in class WindowAssigner<Object,TimeWindow>element - The element to which windows should be assigned.timestamp - The timestamp of the element.context - The WindowAssigner.WindowAssignerContext in which the assigner operates.public Trigger<Object,TimeWindow> getDefaultTrigger(StreamExecutionEnvironment env)
WindowAssignerWindowAssigner.getDefaultTrigger in class WindowAssigner<Object,TimeWindow>public Trigger<Object,TimeWindow> getDefaultTrigger()
WindowAssignerWindowAssigner.
1. If you override getDefaultTrigger(), the getDefaultTrigger() will be
invoked and the getDefaultTrigger(StreamExecutionEnvironment env) won't be invoked.
2. If you don't override getDefaultTrigger(), the getDefaultTrigger(StreamExecutionEnvironment env) will be invoked in the default
implementation of the getDefaultTrigger().
getDefaultTrigger in class WindowAssigner<Object,TimeWindow>@Deprecated public static ProcessingTimeSessionWindows withGap(Time size)
withGap(Duration)SessionWindows WindowAssigner that assigns elements to sessions
based on the element timestamp.size - The session timeout, i.e. the time gap between sessionspublic static ProcessingTimeSessionWindows withGap(Duration size)
SessionWindows WindowAssigner that assigns elements to sessions
based on the element timestamp.size - The session timeout, i.e. the time gap between sessions@PublicEvolving public static <T> DynamicProcessingTimeSessionWindows<T> withDynamicGap(SessionWindowTimeGapExtractor<T> sessionWindowTimeGapExtractor)
SessionWindows WindowAssigner that assigns elements to sessions
based on the element timestamp.sessionWindowTimeGapExtractor - The extractor to use to extract the time gap from the
input elementspublic org.apache.flink.api.common.typeutils.TypeSerializer<TimeWindow> getWindowSerializer(org.apache.flink.api.common.ExecutionConfig executionConfig)
WindowAssignerTypeSerializer for serializing windows that are assigned by this WindowAssigner.getWindowSerializer in class WindowAssigner<Object,TimeWindow>public boolean isEventTime()
WindowAssignertrue if elements are assigned to windows based on event time, false
otherwise.isEventTime in class WindowAssigner<Object,TimeWindow>public void mergeWindows(Collection<TimeWindow> windows, MergingWindowAssigner.MergeCallback<TimeWindow> c)
TimeWindows.mergeWindows in class MergingWindowAssigner<Object,TimeWindow>windows - The window candidates.c - A callback that can be invoked to signal which windows should be merged.Copyright © 2014–2025 The Apache Software Foundation. All rights reserved.