K
- The type of key returned by the KeySelector
.IN
- The type of the incoming elements.OUT
- The type of elements emitted by the InternalWindowFunction
.W
- The type of Window
that the WindowAssigner
assigns.@Internal public class WindowOperator<K,IN,ACC,OUT,W extends Window> extends AbstractUdfStreamOperator<OUT,InternalWindowFunction<ACC,OUT,K,W>> implements OneInputStreamOperator<IN,OUT>, Triggerable<K,W>
WindowAssigner
and
Trigger
.
When an element arrives it gets assigned a key using a KeySelector
and it gets
assigned to zero or more windows using a WindowAssigner
. Based on this, the element
is put into panes. A pane is the bucket of elements that have the same key and same
Window
. An element can be in multiple panes if it was assigned to multiple windows by the
WindowAssigner
.
Each pane gets its own instance of the provided Trigger
. This trigger determines when
the contents of the pane should be processed to emit results. When a trigger fires,
the given InternalWindowFunction
is invoked to produce the results that are emitted for
the pane to which the Trigger
belongs.
Modifier and Type | Class and Description |
---|---|
class |
WindowOperator.AbstractPerWindowStateStore
Base class for per-window
KeyedStateStores . |
class |
WindowOperator.Context
Context is a utility for handling Trigger invocations. |
class |
WindowOperator.MergingWindowStateStore
Special
WindowOperator.AbstractPerWindowStateStore that doesn't allow access to per-window state. |
class |
WindowOperator.PerWindowStateStore
Regular per-window state store for use with
ProcessWindowFunction . |
protected static class |
WindowOperator.Timer<K,W extends Window>
Internal class for keeping track of in-flight timers.
|
class |
WindowOperator.WindowContext
A utility class for handling
ProcessWindowFunction invocations. |
AbstractStreamOperator.CountingOutput<OUT>
Modifier and Type | Field and Description |
---|---|
protected long |
allowedLateness
The allowed lateness for elements.
|
protected InternalTimerService<W> |
internalTimerService |
protected TypeSerializer<K> |
keySerializer
For serializing the key in checkpoints.
|
protected OutputTag<IN> |
lateDataOutputTag
OutputTag to use for late arriving events. |
protected Counter |
numLateRecordsDropped |
protected WindowOperator.WindowContext |
processContext |
protected TimestampedCollector<OUT> |
timestampedCollector
This is given to the
InternalWindowFunction for emitting elements with a given
timestamp. |
protected WindowOperator.Context |
triggerContext |
protected WindowAssigner<? super IN,W> |
windowAssigner |
protected WindowAssigner.WindowAssignerContext |
windowAssignerContext |
protected TypeSerializer<W> |
windowSerializer
For serializing the window in checkpoints.
|
userFunction
chainingStrategy, config, latencyStats, LOG, metrics, output, timeServiceManager
Constructor and Description |
---|
WindowOperator(WindowAssigner<? super IN,W> windowAssigner,
TypeSerializer<W> windowSerializer,
KeySelector<IN,K> keySelector,
TypeSerializer<K> keySerializer,
StateDescriptor<? extends AppendingState<IN,ACC>,?> windowStateDescriptor,
InternalWindowFunction<ACC,OUT,K,W> windowFunction,
Trigger<? super IN,? super W> trigger,
long allowedLateness,
OutputTag<IN> lateDataOutputTag)
Creates a new
WindowOperator based on the given policies and user functions. |
Modifier and Type | Method and Description |
---|---|
void |
close()
This method is called after all records have been added to the operators via the methods
OneInputStreamOperator.processElement(StreamRecord) , or
TwoInputStreamOperator.processElement1(StreamRecord) and
TwoInputStreamOperator.processElement2(StreamRecord) . |
protected void |
deleteCleanupTimer(W window)
Deletes the cleanup timer set for the contents of the provided window.
|
void |
dispose()
This method is called at the very end of the operator's life, both in the case of a successful
completion of the operation, and in the case of a failure and canceling.
|
KeySelector<IN,K> |
getKeySelector() |
protected MergingWindowSet<W> |
getMergingWindowSet()
Retrieves the
MergingWindowSet for the currently active key. |
StateDescriptor<? extends AppendingState<IN,ACC>,?> |
getStateDescriptor() |
Trigger<? super IN,? super W> |
getTrigger() |
WindowAssigner<? super IN,W> |
getWindowAssigner() |
protected boolean |
isCleanupTime(W window,
long time)
Returns
true if the given time is the cleanup time for the given window. |
protected boolean |
isElementLate(StreamRecord<IN> element)
Decide if a record is currently late, based on current watermark and allowed lateness.
|
protected boolean |
isWindowLate(W window)
Returns
true if the watermark is after the end timestamp plus the allowed lateness
of the given window. |
void |
onEventTime(InternalTimer<K,W> timer)
Invoked when an event-time timer fires.
|
void |
onProcessingTime(InternalTimer<K,W> timer)
Invoked when a processing-time timer fires.
|
void |
open()
This method is called immediately before any elements are processed, it should contain the
operator's initialization logic, e.g.
|
void |
processElement(StreamRecord<IN> element)
Processes one element that arrived at this operator.
|
protected void |
registerCleanupTimer(W window)
Registers a timer to cleanup the content of the window.
|
protected void |
sideOutput(StreamRecord<IN> element)
Write skipped late arriving element to SideOutput.
|
getUserFunction, getUserFunctionParameters, initializeState, notifyCheckpointComplete, setOutputType, setup, snapshotState
getChainingStrategy, getContainingTask, getCurrentKey, getExecutionConfig, getInternalTimerService, getKeyedStateBackend, getKeyedStateStore, getMetricGroup, getOperatorConfig, getOperatorID, getOperatorName, getOperatorStateBackend, getOrCreateKeyedState, getPartitionedState, getPartitionedState, getProcessingTimeService, getRuntimeContext, getUserCodeClassloader, initializeState, numEventTimeTimers, numProcessingTimeTimers, prepareSnapshotPreBarrier, processLatencyMarker, processLatencyMarker1, processLatencyMarker2, processWatermark, processWatermark1, processWatermark2, reportOrForwardLatencyMarker, setChainingStrategy, setCurrentKey, setKeyContextElement1, setKeyContextElement2, snapshotState
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
processLatencyMarker, processWatermark
getChainingStrategy, getMetricGroup, getOperatorID, initializeState, prepareSnapshotPreBarrier, setChainingStrategy, setKeyContextElement1, setKeyContextElement2, setup, snapshotState
notifyCheckpointComplete
getCurrentKey, setCurrentKey
protected final WindowAssigner<? super IN,W extends Window> windowAssigner
protected final TypeSerializer<K> keySerializer
protected final TypeSerializer<W extends Window> windowSerializer
protected final long allowedLateness
window.maxTimestamp + allowedLateness
landmark.
protected final OutputTag<IN> lateDataOutputTag
OutputTag
to use for late arriving events. Elements for which
window.maxTimestamp + allowedLateness
is smaller than the current watermark will
be emitted to this.protected transient Counter numLateRecordsDropped
protected transient TimestampedCollector<OUT> timestampedCollector
InternalWindowFunction
for emitting elements with a given
timestamp.protected transient WindowOperator.Context triggerContext
protected transient WindowOperator.WindowContext processContext
protected transient WindowAssigner.WindowAssignerContext windowAssignerContext
protected transient InternalTimerService<W extends Window> internalTimerService
public WindowOperator(WindowAssigner<? super IN,W> windowAssigner, TypeSerializer<W> windowSerializer, KeySelector<IN,K> keySelector, TypeSerializer<K> keySerializer, StateDescriptor<? extends AppendingState<IN,ACC>,?> windowStateDescriptor, InternalWindowFunction<ACC,OUT,K,W> windowFunction, Trigger<? super IN,? super W> trigger, long allowedLateness, OutputTag<IN> lateDataOutputTag)
WindowOperator
based on the given policies and user functions.public void open() throws Exception
AbstractStreamOperator
The default implementation does nothing.
open
in interface StreamOperator<OUT>
open
in class AbstractUdfStreamOperator<OUT,InternalWindowFunction<ACC,OUT,K,W extends Window>>
Exception
- An exception in this method causes the operator to fail.public void close() throws Exception
AbstractStreamOperator
OneInputStreamOperator.processElement(StreamRecord)
, or
TwoInputStreamOperator.processElement1(StreamRecord)
and
TwoInputStreamOperator.processElement2(StreamRecord)
.
The method is expected to flush all remaining buffered data. Exceptions during this flushing of buffered should be propagated, in order to cause the operation to be recognized asa failed, because the last data items are not processed properly.
close
in interface StreamOperator<OUT>
close
in class AbstractUdfStreamOperator<OUT,InternalWindowFunction<ACC,OUT,K,W extends Window>>
Exception
- An exception in this method causes the operator to fail.public void dispose() throws Exception
AbstractStreamOperator
This method is expected to make a thorough effort to release all resources that the operator has acquired.
dispose
in interface StreamOperator<OUT>
dispose
in interface Disposable
dispose
in class AbstractUdfStreamOperator<OUT,InternalWindowFunction<ACC,OUT,K,W extends Window>>
Exception
- if something goes wrong during disposal.public void processElement(StreamRecord<IN> element) throws Exception
OneInputStreamOperator
processElement
in interface OneInputStreamOperator<IN,OUT>
Exception
public void onEventTime(InternalTimer<K,W> timer) throws Exception
Triggerable
onEventTime
in interface Triggerable<K,W extends Window>
Exception
public void onProcessingTime(InternalTimer<K,W> timer) throws Exception
Triggerable
onProcessingTime
in interface Triggerable<K,W extends Window>
Exception
protected void sideOutput(StreamRecord<IN> element)
element
- skipped late arriving element to side outputprotected MergingWindowSet<W> getMergingWindowSet() throws Exception
MergingWindowSet
for the currently active key.
The caller must ensure that the correct key is set in the state backend.
The caller must also ensure to properly persist changes to state using
MergingWindowSet.persist()
.
Exception
protected boolean isWindowLate(W window)
true
if the watermark is after the end timestamp plus the allowed lateness
of the given window.protected boolean isElementLate(StreamRecord<IN> element)
element
- The element to checkprotected void registerCleanupTimer(W window)
window
- the window whose state to discardprotected void deleteCleanupTimer(W window)
window
- the window whose state to discardprotected final boolean isCleanupTime(W window, long time)
true
if the given time is the cleanup time for the given window.@VisibleForTesting public Trigger<? super IN,? super W> getTrigger()
@VisibleForTesting public KeySelector<IN,K> getKeySelector()
@VisibleForTesting public WindowAssigner<? super IN,W> getWindowAssigner()
@VisibleForTesting public StateDescriptor<? extends AppendingState<IN,ACC>,?> getStateDescriptor()
Copyright © 2014–2020 The Apache Software Foundation. All rights reserved.