@PublicEvolving public class TumblingEventTimeWindows extends WindowAssigner<Object,TimeWindow>
WindowAssigner
that windows elements into windows based on the timestamp of the
elements. Windows cannot overlap.
For example, in order to window into windows of 1 minute:
DataStream<Tuple2<String, Integer>> in = ...;
KeyedStream<Tuple2<String, Integer>, String> keyed = in.keyBy(...);
WindowedStream<Tuple2<String, Integer>, String, TimeWindow> windowed =
keyed.window(TumblingEventTimeWindows.of(Duration.ofMinutes(1)));
WindowAssigner.WindowAssignerContext
Modifier | Constructor and Description |
---|---|
protected |
TumblingEventTimeWindows(long size,
long offset,
WindowStagger windowStagger) |
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 . |
TypeSerializer<TimeWindow> |
getWindowSerializer(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. |
static TumblingEventTimeWindows |
of(Duration size)
Creates a new
TumblingEventTimeWindows WindowAssigner that assigns elements
to time windows based on the element timestamp. |
static TumblingEventTimeWindows |
of(Duration size,
Duration offset)
Creates a new
TumblingEventTimeWindows WindowAssigner that assigns elements
to time windows based on the element timestamp and offset. |
static TumblingEventTimeWindows |
of(Duration size,
Duration offset,
WindowStagger windowStagger)
Creates a new
TumblingEventTimeWindows WindowAssigner that assigns elements
to time windows based on the element timestamp, offset and a staggering offset, depending on
the staggering policy. |
static TumblingEventTimeWindows |
of(Time size)
Deprecated.
Use
of(Duration) |
static TumblingEventTimeWindows |
of(Time size,
Time offset)
Deprecated.
|
static TumblingEventTimeWindows |
of(Time size,
Time offset,
WindowStagger windowStagger)
Deprecated.
|
String |
toString() |
protected TumblingEventTimeWindows(long size, long offset, WindowStagger windowStagger)
public Collection<TimeWindow> assignWindows(Object element, long timestamp, WindowAssigner.WindowAssignerContext context)
WindowAssigner
Collection
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)
WindowAssigner
WindowAssigner
.getDefaultTrigger
in class WindowAssigner<Object,TimeWindow>
public Trigger<Object,TimeWindow> getDefaultTrigger()
WindowAssigner
WindowAssigner
.
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 TumblingEventTimeWindows of(Time size)
of(Duration)
TumblingEventTimeWindows
WindowAssigner
that assigns elements
to time windows based on the element timestamp.size
- The size of the generated windows.public static TumblingEventTimeWindows of(Duration size)
TumblingEventTimeWindows
WindowAssigner
that assigns elements
to time windows based on the element timestamp.size
- The size of the generated windows.@Deprecated public static TumblingEventTimeWindows of(Time size, Time offset)
of(Duration, Duration)
TumblingEventTimeWindows
WindowAssigner
that assigns elements
to time windows based on the element timestamp and offset.
For example, if you want window a stream by hour,but window begins at the 15th minutes of
each hour, you can use of(Time.hours(1),Time.minutes(15))
,then you will get time
windows start at 0:15:00,1:15:00,2:15:00,etc.
Rather than that,if you are living in somewhere which is not using UTC±00:00 time, such as
China which is using UTC+08:00,and you want a time window with size of one day, and window
begins at every 00:00:00 of local time,you may use of(Time.days(1),Time.hours(-8))
.
The parameter of offset is Time.hours(-8))
since UTC+08:00 is 8 hours earlier than
UTC time.
size
- The size of the generated windows.offset
- The offset which window start would be shifted by.public static TumblingEventTimeWindows of(Duration size, Duration offset)
TumblingEventTimeWindows
WindowAssigner
that assigns elements
to time windows based on the element timestamp and offset.
For example, if you want window a stream by hour,but window begins at the 15th minutes of
each hour, you can use of(Duration.ofHours(1), Duration.ofMinutes(15))
, then you will
get time windows start at 0:15:00,1:15:00,2:15:00,etc.
Rather than that,if you are living in somewhere which is not using UTC±00:00 time, such as
China which is using UTC+08:00,and you want a time window with size of one day, and window
begins at every 00:00:00 of local time,you may use of(Duration.ofDays(1),
Duration.ofHours(-8))
. The parameter of offset is Duration.ofHours(-8))
since
UTC+08:00 is 8 hours earlier than UTC time.
size
- The size of the generated windows.offset
- The offset which window start would be shifted by.@Deprecated @PublicEvolving public static TumblingEventTimeWindows of(Time size, Time offset, WindowStagger windowStagger)
of(Duration, Duration, WindowStagger)
TumblingEventTimeWindows
WindowAssigner
that assigns elements
to time windows based on the element timestamp, offset and a staggering offset, depending on
the staggering policy.size
- The size of the generated windows.offset
- The globalOffset which window start would be shifted by.windowStagger
- The utility that produces staggering offset in runtime.@PublicEvolving public static TumblingEventTimeWindows of(Duration size, Duration offset, WindowStagger windowStagger)
TumblingEventTimeWindows
WindowAssigner
that assigns elements
to time windows based on the element timestamp, offset and a staggering offset, depending on
the staggering policy.size
- The size of the generated windows.offset
- The globalOffset which window start would be shifted by.windowStagger
- The utility that produces staggering offset in runtime.public TypeSerializer<TimeWindow> getWindowSerializer(ExecutionConfig executionConfig)
WindowAssigner
TypeSerializer
for serializing windows that are assigned by this WindowAssigner
.getWindowSerializer
in class WindowAssigner<Object,TimeWindow>
public boolean isEventTime()
WindowAssigner
true
if elements are assigned to windows based on event time, false
otherwise.isEventTime
in class WindowAssigner<Object,TimeWindow>
Copyright © 2014–2024 The Apache Software Foundation. All rights reserved.