java.lang.Object
org.apache.flink.runtime.asyncprocessing.operators.windowing.triggers.AsyncTrigger<T,W>
Type Parameters:
T - The type of elements on which this Trigger works.
W - The type of Windows on which this Trigger can operate.
All Implemented Interfaces:
Serializable
Direct Known Subclasses:
AsyncContinuousEventTimeTrigger, AsyncCountTrigger, AsyncEventTimeTrigger, AsyncProcessingTimeTrigger, AsyncPurgingTrigger

@Internal public abstract class AsyncTrigger<T,W extends Window> extends Object implements Serializable
A Trigger determines when a pane of a window should be evaluated to emit the results for that part of the window. This trigger is for the async window operator, i.e., AsyncWindowOperator.

All State APIs in the async trigger are from State V2.

A pane is the bucket of elements that have the same key (assigned by the KeySelector) and same Window. An element can be in multiple panes if it was assigned to multiple windows by the WindowAssigner. These panes all have their own instance of the Trigger.

Triggers must not maintain state internally since they can be re-created or reused for different keys. All necessary state should be persisted using the state abstraction available on the AsyncTrigger.TriggerContext.

When used with a MergingWindowAssigner the Trigger must return true from canMerge() and onMerge(Window, OnMergeContext) most be properly implemented.

See Also:
  • Constructor Details

    • AsyncTrigger

      public AsyncTrigger()
  • Method Details

    • onElement

      public abstract org.apache.flink.api.common.state.v2.StateFuture<TriggerResult> onElement(T element, long timestamp, W window, AsyncTrigger.TriggerContext ctx) throws Exception
      Called for every element that gets added to a pane. The result of this will determine whether the pane is evaluated to emit results.
      Parameters:
      element - The element that arrived.
      timestamp - The timestamp of the element that arrived.
      window - The window to which the element is being added.
      ctx - A context object that can be used to register timer callbacks.
      Throws:
      Exception
    • onProcessingTime

      public abstract org.apache.flink.api.common.state.v2.StateFuture<TriggerResult> onProcessingTime(long time, W window, AsyncTrigger.TriggerContext ctx) throws Exception
      Called when a processing-time timer that was set using the trigger context fires.
      Parameters:
      time - The timestamp at which the timer fired.
      window - The window for which the timer fired.
      ctx - A context object that can be used to register timer callbacks.
      Throws:
      Exception
    • onEventTime

      public abstract org.apache.flink.api.common.state.v2.StateFuture<TriggerResult> onEventTime(long time, W window, AsyncTrigger.TriggerContext ctx) throws Exception
      Called when an event-time timer that was set using the trigger context fires.
      Parameters:
      time - The timestamp at which the timer fired.
      window - The window for which the timer fired.
      ctx - A context object that can be used to register timer callbacks.
      Throws:
      Exception
    • canMerge

      public boolean canMerge()
      Returns true if this trigger supports merging of trigger state and can therefore be used with a MergingWindowAssigner.

      If this returns true you must properly implement onMerge(Window, OnMergeContext)

    • onMerge

      public void onMerge(W window, AsyncTrigger.OnMergeContext ctx) throws Exception
      Called when several windows have been merged into one window by the WindowAssigner.
      Parameters:
      window - The new window that results from the merge.
      ctx - A context object that can be used to register timer callbacks and access state.
      Throws:
      Exception
    • clear

      public abstract org.apache.flink.api.common.state.v2.StateFuture<Void> clear(W window, AsyncTrigger.TriggerContext ctx) throws Exception
      Clears any state that the trigger might still hold for the given window. This is called when a window is purged. Timers set using AsyncTrigger.TriggerContext.registerEventTimeTimer(long) and AsyncTrigger.TriggerContext.registerProcessingTimeTimer(long) should be deleted here as well as state acquired using AsyncTrigger.TriggerContext.getPartitionedState(StateDescriptor).
      Throws:
      Exception
    • isEndOfStreamTrigger

      public boolean isEndOfStreamTrigger()
      Indicate whether the trigger only trigger at the end of stream.