Class PythonStreamGroupWindowAggregateOperator<K,W extends Window>
- java.lang.Object
-
- org.apache.flink.streaming.api.operators.AbstractStreamOperator<OUT>
-
- org.apache.flink.streaming.api.operators.python.AbstractPythonFunctionOperator<OUT>
-
- org.apache.flink.streaming.api.operators.python.process.AbstractExternalPythonFunctionOperator<OUT>
-
- org.apache.flink.table.runtime.operators.python.AbstractOneInputPythonFunctionOperator<RowData,RowData>
-
- org.apache.flink.table.runtime.operators.python.aggregate.AbstractPythonStreamAggregateOperator
-
- org.apache.flink.table.runtime.operators.python.aggregate.PythonStreamGroupWindowAggregateOperator<K,W>
-
- All Implemented Interfaces:
Serializable
,CheckpointListener
,BoundedOneInput
,Input<RowData>
,KeyContext
,KeyContextHandler
,OneInputStreamOperator<RowData,RowData>
,StreamOperator<RowData>
,StreamOperatorStateHandler.CheckpointedStreamOperator
,Triggerable<K,W>
,YieldingOperator<RowData>
@Internal public class PythonStreamGroupWindowAggregateOperator<K,W extends Window> extends AbstractPythonStreamAggregateOperator implements Triggerable<K,W>
The Python Group Window AggregateFunction operator.- See Also:
- Serialized Form
-
-
Field Summary
-
Fields inherited from class org.apache.flink.table.runtime.operators.python.aggregate.AbstractPythonStreamAggregateOperator
bais, baisWrapper, baos, baosWrapper, inputType, outputType, rowDataWrapper, userDefinedFunctionInputType, userDefinedFunctionOutputType
-
Fields inherited from class org.apache.flink.streaming.api.operators.python.process.AbstractExternalPythonFunctionOperator
pythonFunctionRunner
-
Fields inherited from class org.apache.flink.streaming.api.operators.python.AbstractPythonFunctionOperator
bundleFinishedCallback, config, elementCount, lastFinishBundleTime, maxBundleSize, systemEnvEnabled
-
Fields inherited from class org.apache.flink.streaming.api.operators.AbstractStreamOperator
combinedWatermark, lastRecordAttributes1, lastRecordAttributes2, latencyStats, LOG, metrics, output, processingTimeService, stateHandler, stateKeySelector1, stateKeySelector2, timeServiceManager
-
-
Constructor Summary
Constructors Modifier Constructor Description protected
PythonStreamGroupWindowAggregateOperator(Configuration config, RowType inputType, RowType outputType, PythonAggregateFunctionInfo[] aggregateFunctions, DataViewSpec[][] dataViewSpecs, int[] grouping, int indexOfCountStar, boolean generateUpdateBefore, boolean countStarInserted, int inputTimeFieldIndex, GroupWindowAssigner<W> windowAssigner, FlinkFnApi.GroupWindow.WindowType windowType, boolean isRowTime, boolean isTimeWindow, long size, long slide, long gap, long allowedLateness, NamedWindowProperty[] namedProperties, ZoneId shiftTimeZone)
-
Method Summary
All Methods Static Methods Instance Methods Concrete Methods Modifier and Type Method Description FlinkFnApi.CoderInfoDescriptor
createInputCoderInfoDescriptor(RowType runnerInputType)
FlinkFnApi.CoderInfoDescriptor
createOutputCoderInfoDescriptor(RowType runnerOutType)
static <K,W extends Window>
PythonStreamGroupWindowAggregateOperator<K,W>createSessionGroupWindowAggregateOperator(Configuration config, RowType inputType, RowType outputType, PythonAggregateFunctionInfo[] aggregateFunctions, DataViewSpec[][] dataViewSpecs, int[] grouping, int indexOfCountStar, boolean generateUpdateBefore, boolean countStarInserted, int inputTimeFieldIndex, GroupWindowAssigner<W> windowAssigner, boolean isRowTime, long gap, long allowedLateness, NamedWindowProperty[] namedProperties, ZoneId shiftTimeZone)
static <K,W extends Window>
PythonStreamGroupWindowAggregateOperator<K,W>createSlidingGroupWindowAggregateOperator(Configuration config, RowType inputType, RowType outputType, PythonAggregateFunctionInfo[] aggregateFunctions, DataViewSpec[][] dataViewSpecs, int[] grouping, int indexOfCountStar, boolean generateUpdateBefore, boolean countStarInserted, int inputTimeFieldIndex, GroupWindowAssigner<W> windowAssigner, boolean isRowTime, boolean isTimeWindow, long size, long slide, long allowedLateness, NamedWindowProperty[] namedProperties, ZoneId shiftTimeZone)
static <K,W extends Window>
PythonStreamGroupWindowAggregateOperator<K,W>createTumblingGroupWindowAggregateOperator(Configuration config, RowType inputType, RowType outputType, PythonAggregateFunctionInfo[] aggregateFunctions, DataViewSpec[][] dataViewSpecs, int[] grouping, int indexOfCountStar, boolean generateUpdateBefore, boolean countStarInserted, int inputTimeFieldIndex, GroupWindowAssigner<W> windowAssigner, boolean isRowTime, boolean isTimeWindow, long size, long allowedLateness, NamedWindowProperty[] namedProperties, ZoneId shiftTimeZone)
RowType
createUserDefinedFunctionInputType()
RowType
createUserDefinedFunctionOutputType()
void
emitResult(Tuple3<String,byte[],Integer> resultTuple)
Sends the execution result to the downstream operator.String
getFunctionUrn()
protected FlinkFnApi.UserDefinedAggregateFunctions
getUserDefinedFunctionsProto()
Gets the proto representation of the Python user-defined aggregate functions to be executed.TypeSerializer<W>
getWindowSerializer()
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. state initialization.void
processElementInternal(RowData value)
-
Methods inherited from class org.apache.flink.table.runtime.operators.python.aggregate.AbstractPythonStreamAggregateOperator
createPythonFunctionRunner, getCurrentKey, getKeyType, getPythonEnv, processElement, setCurrentKey
-
Methods inherited from class org.apache.flink.table.runtime.operators.python.AbstractOneInputPythonFunctionOperator
endInput
-
Methods inherited from class org.apache.flink.streaming.api.operators.python.process.AbstractExternalPythonFunctionOperator
close, createPythonEnvironmentManager, emitResults, invokeFinishBundle
-
Methods inherited from class org.apache.flink.streaming.api.operators.python.AbstractPythonFunctionOperator
checkInvokeFinishBundleByCount, finish, getConfiguration, getFlinkMetricContainer, isBundleFinished, prepareSnapshotPreBarrier, processWatermark
-
Methods inherited from class org.apache.flink.streaming.api.operators.AbstractStreamOperator
getContainingTask, getExecutionConfig, getInternalTimerService, getKeyedStateBackend, getKeyedStateStore, getMetricGroup, getOperatorConfig, getOperatorID, getOperatorName, getOperatorStateBackend, getOrCreateKeyedState, getPartitionedState, getPartitionedState, getProcessingTimeService, getRuntimeContext, getStateKeySelector1, getStateKeySelector2, getTimeServiceManager, getUserCodeClassloader, hasKeyContext1, hasKeyContext2, initializeState, initializeState, isAsyncStateProcessingEnabled, isUsingCustomRawKeyedState, notifyCheckpointAborted, notifyCheckpointComplete, processLatencyMarker, processLatencyMarker1, processLatencyMarker2, processRecordAttributes, processRecordAttributes1, processRecordAttributes2, processWatermark1, processWatermark2, processWatermarkStatus, processWatermarkStatus, processWatermarkStatus1, processWatermarkStatus2, reportOrForwardLatencyMarker, setKeyContextElement1, setKeyContextElement2, setMailboxExecutor, setProcessingTimeService, setup, snapshotState, snapshotState, useSplittableTimers
-
Methods inherited from class java.lang.Object
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
-
Methods inherited from interface org.apache.flink.api.common.state.CheckpointListener
notifyCheckpointAborted, notifyCheckpointComplete
-
Methods inherited from interface org.apache.flink.streaming.api.operators.Input
processLatencyMarker, processRecordAttributes, processWatermark, processWatermarkStatus
-
Methods inherited from interface org.apache.flink.streaming.api.operators.KeyContextHandler
hasKeyContext
-
Methods inherited from interface org.apache.flink.streaming.api.operators.OneInputStreamOperator
setKeyContextElement
-
Methods inherited from interface org.apache.flink.streaming.api.operators.StreamOperator
close, finish, getMetricGroup, getOperatorAttributes, getOperatorID, initializeState, prepareSnapshotPreBarrier, setKeyContextElement1, setKeyContextElement2, snapshotState
-
-
-
-
Constructor Detail
-
PythonStreamGroupWindowAggregateOperator
protected PythonStreamGroupWindowAggregateOperator(Configuration config, RowType inputType, RowType outputType, PythonAggregateFunctionInfo[] aggregateFunctions, DataViewSpec[][] dataViewSpecs, int[] grouping, int indexOfCountStar, boolean generateUpdateBefore, boolean countStarInserted, int inputTimeFieldIndex, GroupWindowAssigner<W> windowAssigner, FlinkFnApi.GroupWindow.WindowType windowType, boolean isRowTime, boolean isTimeWindow, long size, long slide, long gap, long allowedLateness, NamedWindowProperty[] namedProperties, ZoneId shiftTimeZone)
-
-
Method Detail
-
createTumblingGroupWindowAggregateOperator
public static <K,W extends Window> PythonStreamGroupWindowAggregateOperator<K,W> createTumblingGroupWindowAggregateOperator(Configuration config, RowType inputType, RowType outputType, PythonAggregateFunctionInfo[] aggregateFunctions, DataViewSpec[][] dataViewSpecs, int[] grouping, int indexOfCountStar, boolean generateUpdateBefore, boolean countStarInserted, int inputTimeFieldIndex, GroupWindowAssigner<W> windowAssigner, boolean isRowTime, boolean isTimeWindow, long size, long allowedLateness, NamedWindowProperty[] namedProperties, ZoneId shiftTimeZone)
-
createSlidingGroupWindowAggregateOperator
public static <K,W extends Window> PythonStreamGroupWindowAggregateOperator<K,W> createSlidingGroupWindowAggregateOperator(Configuration config, RowType inputType, RowType outputType, PythonAggregateFunctionInfo[] aggregateFunctions, DataViewSpec[][] dataViewSpecs, int[] grouping, int indexOfCountStar, boolean generateUpdateBefore, boolean countStarInserted, int inputTimeFieldIndex, GroupWindowAssigner<W> windowAssigner, boolean isRowTime, boolean isTimeWindow, long size, long slide, long allowedLateness, NamedWindowProperty[] namedProperties, ZoneId shiftTimeZone)
-
createSessionGroupWindowAggregateOperator
public static <K,W extends Window> PythonStreamGroupWindowAggregateOperator<K,W> createSessionGroupWindowAggregateOperator(Configuration config, RowType inputType, RowType outputType, PythonAggregateFunctionInfo[] aggregateFunctions, DataViewSpec[][] dataViewSpecs, int[] grouping, int indexOfCountStar, boolean generateUpdateBefore, boolean countStarInserted, int inputTimeFieldIndex, GroupWindowAssigner<W> windowAssigner, boolean isRowTime, long gap, long allowedLateness, NamedWindowProperty[] namedProperties, ZoneId shiftTimeZone)
-
open
public void open() throws Exception
Description copied from class:AbstractStreamOperator
This method is called immediately before any elements are processed, it should contain the operator's initialization logic, e.g. state initialization.The default implementation does nothing.
- Specified by:
open
in interfaceStreamOperator<K>
- Overrides:
open
in classAbstractPythonStreamAggregateOperator
- Throws:
Exception
- An exception in this method causes the operator to fail.
-
processElementInternal
public void processElementInternal(RowData value) throws Exception
- Specified by:
processElementInternal
in classAbstractPythonStreamAggregateOperator
- Throws:
Exception
-
emitResult
public void emitResult(Tuple3<String,byte[],Integer> resultTuple) throws Exception
Description copied from class:AbstractExternalPythonFunctionOperator
Sends the execution result to the downstream operator.- Specified by:
emitResult
in classAbstractExternalPythonFunctionOperator<RowData>
- Throws:
Exception
-
getFunctionUrn
public String getFunctionUrn()
- Specified by:
getFunctionUrn
in classAbstractPythonStreamAggregateOperator
-
createUserDefinedFunctionInputType
public RowType createUserDefinedFunctionInputType()
- Specified by:
createUserDefinedFunctionInputType
in classAbstractPythonStreamAggregateOperator
-
createUserDefinedFunctionOutputType
public RowType createUserDefinedFunctionOutputType()
- Specified by:
createUserDefinedFunctionOutputType
in classAbstractPythonStreamAggregateOperator
-
getUserDefinedFunctionsProto
protected FlinkFnApi.UserDefinedAggregateFunctions getUserDefinedFunctionsProto()
Description copied from class:AbstractPythonStreamAggregateOperator
Gets the proto representation of the Python user-defined aggregate functions to be executed.- Overrides:
getUserDefinedFunctionsProto
in classAbstractPythonStreamAggregateOperator
-
getWindowSerializer
public TypeSerializer<W> getWindowSerializer()
-
onEventTime
public void onEventTime(InternalTimer<K,W> timer) throws Exception
Description copied from interface:Triggerable
Invoked when an event-time timer fires.- Specified by:
onEventTime
in interfaceTriggerable<K,W extends Window>
- Throws:
Exception
-
onProcessingTime
public void onProcessingTime(InternalTimer<K,W> timer) throws Exception
Description copied from interface:Triggerable
Invoked when a processing-time timer fires.- Specified by:
onProcessingTime
in interfaceTriggerable<K,W extends Window>
- Throws:
Exception
-
createInputCoderInfoDescriptor
public FlinkFnApi.CoderInfoDescriptor createInputCoderInfoDescriptor(RowType runnerInputType)
- Overrides:
createInputCoderInfoDescriptor
in classAbstractPythonStreamAggregateOperator
-
createOutputCoderInfoDescriptor
public FlinkFnApi.CoderInfoDescriptor createOutputCoderInfoDescriptor(RowType runnerOutType)
- Overrides:
createOutputCoderInfoDescriptor
in classAbstractPythonStreamAggregateOperator
-
-