Class | Description |
---|---|
AfterAll |
A composite
Trigger that fires when all of its sub-triggers are ready. |
AfterEach |
A composite
Trigger that executes its sub-triggers in order. |
AfterFirst |
A composite
Trigger that fires once after at least one of its sub-triggers have fired. |
AfterPane |
A
Trigger that fires at some point after a specified number of input elements have
arrived. |
AfterProcessingTime |
A
Trigger trigger that fires at a specified point in processing time, relative to when
input first arrives. |
AfterSynchronizedProcessingTime |
FOR INTERNAL USE ONLY.
|
AfterWatermark |
AfterWatermark triggers fire based on progress of the system watermark. |
AfterWatermark.AfterWatermarkEarlyAndLate | |
AfterWatermark.FromEndOfWindow |
A watermark trigger targeted relative to the end of the window.
|
BoundedWindow |
A
BoundedWindow represents window information assigned to data elements. |
CalendarWindows |
A collection of
WindowFn s that windows values into calendar-based
windows such as spans of days, months, or years. |
CalendarWindows.DaysWindows |
A
WindowFn that windows elements into periods measured by days. |
CalendarWindows.MonthsWindows |
A
WindowFn that windows elements into periods measured by months. |
CalendarWindows.YearsWindows |
A
WindowFn that windows elements into periods measured by years. |
DefaultTrigger |
A trigger that is equivalent to
Repeatedly.forever(AfterWatermark.pastEndOfWindow()) . |
FixedWindows |
A
WindowFn that windows values into fixed-size timestamp-based windows. |
GlobalWindow |
The default window into which all data is placed (via
GlobalWindows ). |
GlobalWindow.Coder |
GlobalWindow.Coder for encoding and decoding GlobalWindow s. |
GlobalWindows |
A
WindowFn that assigns all data to the same window. |
IntervalWindow |
An implementation of
BoundedWindow that represents an interval from
IntervalWindow.start (inclusive) to IntervalWindow.end (exclusive). |
IntervalWindow.IntervalWindowCoder |
Encodes an
IntervalWindow as a pair of its upper bound and duration. |
InvalidWindows<W extends BoundedWindow> |
A
WindowFn that represents an invalid pipeline state. |
MergeOverlappingIntervalWindows |
For internal use only; no backwards compatibility guarantees.
|
Never |
A
Trigger which never fires. |
Never.NeverTrigger |
The actual trigger class for
Never triggers. |
NonMergingWindowFn<T,W extends BoundedWindow> |
Abstract base class for
WindowFns that do not merge windows. |
OrFinallyTrigger |
A
Trigger that executes according to its main trigger until its "finally" trigger fires. |
PaneInfo |
Provides information about the pane an element belongs to.
|
PaneInfo.PaneInfoCoder |
A Coder for encoding PaneInfo instances.
|
PartitioningWindowFn<T,W extends BoundedWindow> |
A
WindowFn that places each value into exactly one window based on its timestamp and
never merges windows. |
Repeatedly |
A
Trigger that fires according to its subtrigger forever. |
ReshuffleTrigger<W extends BoundedWindow> | Deprecated
The intended side effect of
Reshuffle is not portable; it will likely be
removed |
Sessions |
A
WindowFn that windows values into sessions separated by periods with no input for at
least the duration specified by Sessions.getGapDuration() . |
SlidingWindows |
A
WindowFn that windows values into possibly overlapping fixed-size
timestamp-based windows. |
TimestampTransform |
For internal use only; no backwards-compatibility guarantees.
|
TimestampTransform.AlignTo |
For internal use only; no backwards-compatibility guarantees.
|
TimestampTransform.Delay |
For internal use only; no backwards-compatibility guarantees.
|
Trigger |
Triggers control when the elements for a specific key and window are output. |
Trigger.OnceTrigger |
For internal use only; no backwards-compatibility guarantees.
Triggers that are guaranteed to fire at most once should extend Trigger.OnceTrigger rather than the general Trigger class to indicate that behavior. |
Window<T> |
Window logically divides up or groups the elements of a
PCollection into finite windows according to a WindowFn . |
Window.Assign<T> |
A Primitive
PTransform that assigns windows to elements based on a WindowFn . |
WindowFn<T,W extends BoundedWindow> |
The argument to the
Window transform used to assign elements into
windows and to determine how windows are merged. |
WindowMappingFn<TargetWindowT extends BoundedWindow> |
Experimental! This will be ready for users eventually, but should be considered internal for
now.
|
Enum | Description |
---|---|
PaneInfo.Timing |
Enumerates the possibilities for the timing of this pane firing related to the
input and output watermarks for its computation.
|
TimestampCombiner |
Policies for combining timestamps that occur within a window.
|
Window.ClosingBehavior |
Specifies the conditions under which a final pane will be created when a window is permanently
closed.
|
Window.OnTimeBehavior |
Specifies the conditions under which an on-time pane will be created when a window is closed.
|
Exception | Description |
---|---|
IncompatibleWindowException |
Exception thrown by
WindowFn.verifyCompatibility(WindowFn) if two compared
WindowFns are not compatible, including the explanation of incompatibility. |
Window
transform
for dividing the elements in a PCollection into windows, and the
Trigger
for controlling when those
elements are output.
Window
logically divides up or groups the elements of a
PCollection
into finite windows according to a
WindowFn
.
The output of Window
contains the same elements as input, but they
have been logically assigned to windows. The next
GroupByKey
s, including one
within composite transforms, will group by the combination of keys and
windows.
Windowing a PCollection
allows chunks of it to be processed
individually, before the entire PCollection
is available. This is
especially important for PCollection
s with unbounded size, since the full
PCollection
is never available at once.
For PCollection
s with a bounded size, by default, all data is implicitly in a
single window, and this replicates conventional batch mode. However, windowing can still be a
convenient way to express time-sliced algorithms over bounded PCollection
s.
As elements are assigned to a window, they are are placed into a pane. When the trigger fires all of the elements in the current pane are output.
The DefaultTrigger
will output a
window when the system watermark passes the end of the window. See
AfterWatermark
for details on the
watermark.