@DefaultAnnotation(value=org.checkerframework.checker.nullness.qual.NonNull.class)
See: Description
| Class | Description | 
|---|---|
| AfterAll | A composite  Triggerthat fires when all of its sub-triggers are ready. | 
| AfterEach | A composite  Triggerthat executes its sub-triggers in order. | 
| AfterFirst | A composite  Triggerthat fires once after at least one of its sub-triggers have fired. | 
| AfterPane | A  Triggerthat fires at some point after a specified number of input elements have
 arrived. | 
| AfterProcessingTime | A  Triggertrigger that fires at a specified point in processing time, relative to when
 input first arrives. | 
| AfterSynchronizedProcessingTime | FOR INTERNAL USE ONLY. | 
| AfterWatermark | AfterWatermarktriggers 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  BoundedWindowrepresents window information assigned to data elements. | 
| CalendarWindows | A collection of  WindowFns that windows values into calendar-based windows such as spans
 of days, months, or years. | 
| CalendarWindows.DaysWindows | A  WindowFnthat windows elements into periods measured by days. | 
| CalendarWindows.MonthsWindows | A  WindowFnthat windows elements into periods measured by months. | 
| CalendarWindows.YearsWindows | A  WindowFnthat windows elements into periods measured by years. | 
| DefaultTrigger | A trigger that is equivalent to  Repeatedly.forever(AfterWatermark.pastEndOfWindow()). | 
| FixedWindows | A  WindowFnthat windows values into fixed-size timestamp-based windows. | 
| GlobalWindow | The default window into which all data is placed (via  GlobalWindows). | 
| GlobalWindow.Coder | GlobalWindow.Coderfor encoding and decodingGlobalWindows. | 
| GlobalWindows | A  WindowFnthat assigns all data to the same window. | 
| IntervalWindow | An implementation of  BoundedWindowthat represents an interval fromIntervalWindow.start(inclusive) toIntervalWindow.end(exclusive). | 
| IntervalWindow.IntervalWindowCoder | Encodes an  IntervalWindowas a pair of its upper bound and duration. | 
| MergeOverlappingIntervalWindows | For internal use only; no backwards compatibility guarantees. | 
| Never | A  Triggerwhich never fires. | 
| Never.NeverTrigger | The actual trigger class for  Nevertriggers. | 
| NonMergingWindowFn<T,W extends BoundedWindow> | Abstract base class for  WindowFnsthat do not merge windows. | 
| OrFinallyTrigger | A  Triggerthat 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  WindowFnthat places each value into exactly one window based on its timestamp and
 never merges windows. | 
| Repeatedly | A  Triggerthat fires according to its subtrigger forever. | 
| ReshuffleTrigger<W extends BoundedWindow> | For internal use only; no backwards compatibility guarantees. | 
| Sessions | A  WindowFnthat windows values into sessions separated by periods with no input for at
 least the duration specified bySessions.getGapDuration(). | 
| SlidingWindows | A  WindowFnthat 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. | 
| Window<T> | Windowlogically divides up or groups the elements of aPCollectioninto finite
 windows according to aWindowFn. | 
| Window.Assign<T> | A Primitive  PTransformthat assigns windows to elements based on aWindowFn. | 
| WindowFn<T,W extends BoundedWindow> | The argument to the  Windowtransform used to assign elements into windows and to
 determine how windows are merged. | 
| WindowMappingFn<TargetWindowT extends BoundedWindow> | A function that takes the windows of elements in a main input and maps them to the appropriate
 window in a  PCollectionViewconsumed as aside input. | 
| 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 GroupByKeys, 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 PCollections
 with unbounded size, since the full PCollection is never available at once.
 
For PCollections 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 PCollections.
 
As elements are assigned to a window, they 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.