Interface RichFunction
-
- All Superinterfaces:
Function
,Serializable
- All Known Subinterfaces:
FilterCondition
,JoinCondition
- All Known Implementing Classes:
AbstractRichFunction
,AbstractRowTimeUnboundedPrecedingOver
,AbstractTopNFunction
,AggregateApplyAllWindowFunction
,AggregateApplyWindowFunction
,AggregateEvictingWindowReaderFunction
,AppendOnlyFirstNFunction
,AppendOnlyTopNFunction
,ArrowSourceFunction
,ArtificalOperatorStateMapper
,ArtificialKeyedStateMapper
,AssignRangeIndex
,AsyncFunctionRunner
,AsyncLookupJoinRunner
,AsyncLookupJoinWithCalcRunner
,BaseBroadcastProcessFunction
,BatchPartitionCommitterSink
,BoundedOutOfOrderWatermarkGenerator
,BroadcastProcessFunction
,BroadcastStateBootstrapFunction
,BulkIterationBase.TerminationCriterionMapper
,CollectSink
,CollectSinkFunction
,ContinuousFileMonitoringFunction
,CoProcessFunction
,DataGeneratorSource
,EvictingWindowReaderFunction
,FailureMapper
,FastTop1Function
,FromSplittableIteratorFunction
,GroupAggFunction
,GroupTableAggFunction
,HadoopMapFunction
,HadoopReducerWrappedFunction
,InputFormatSourceFunction
,InternalAggregateProcessAllWindowFunction
,InternalAggregateProcessWindowFunction
,InternalIterableAllWindowFunction
,InternalIterableProcessAllWindowFunction
,InternalIterableProcessWindowFunction
,InternalIterableWindowFunction
,InternalSingleValueAllWindowFunction
,InternalSingleValueProcessAllWindowFunction
,InternalSingleValueProcessWindowFunction
,InternalSingleValueWindowFunction
,IntervalJoinFunction
,IterativeConditionRunner
,JoinConditionWithNullFilters
,KeyedBroadcastProcessFunction
,KeyedCoProcessFunction
,KeyedLookupJoinWrapper
,KeyedProcessFunction
,KeyedProcessFunctionWithCleanupState
,KeyedStateBootstrapFunction
,KeyedStateReaderFunction
,ListenableCollector
,LookupJoinRunner
,LookupJoinWithCalcRunner
,NoOpFunction
,OperatorSubtaskStateReducer
,OutputFormatSinkFunction
,PassThroughReader
,PatternFlatSelectAdapter
,PatternProcessFunction
,PatternProcessFunctionRunner
,PatternSelectAdapter
,PatternTimeoutFlatSelectAdapter
,PatternTimeoutSelectAdapter
,PrintSinkFunction
,ProcessAllWindowFunction
,ProcessEvictingWindowReader
,ProcessFunction
,ProcessJoinFunction
,ProcessWindowFunction
,ProcTimeDeduplicateKeepFirstRowFunction
,ProcTimeDeduplicateKeepLastRowFunction
,ProcTimeIntervalJoin
,ProcTimeRangeBoundedPrecedingFunction
,ProcTimeRowsBoundedPrecedingFunction
,ProcTimeUnboundedPrecedingFunction
,PythonConnectorUtils.RowRowMapper
,RangeBoundaryBuilder
,ReduceApplyAllWindowFunction
,ReduceApplyProcessAllWindowFunction
,ReduceApplyProcessWindowFunction
,ReduceApplyWindowFunction
,ReduceEvictingWindowReaderFunction
,RetractableTopNFunction
,RichAggregateFunction
,RichAllWindowFunction
,RichAndCondition
,RichAsyncFunction
,RichCoFlatMapFunction
,RichCoGroupFunction
,RichCoMapFunction
,RichCompositeIterativeCondition
,RichCrossFunction
,RichFilterFunction
,RichFlatJoinFunction
,RichFlatMapFunction
,RichGroupCombineFunction
,RichGroupReduceFunction
,RichIterativeCondition
,RichJoinFunction
,RichMapFunction
,RichMapPartitionFunction
,RichNotCondition
,RichOrCondition
,RichParallelSourceFunction
,RichPatternFlatSelectFunction
,RichPatternSelectFunction
,RichReduceFunction
,RichSinkFunction
,RichSourceFunction
,RichWindowFunction
,RowTimeDeduplicateFunction
,RowTimeIntervalJoin
,RowTimeRangeBoundedPrecedingFunction
,RowTimeRangeUnboundedPrecedingFunction
,RowTimeRowsBoundedPrecedingFunction
,RowTimeRowsUnboundedPrecedingFunction
,SemanticsCheckMapper
,SequenceGeneratorSource
,SideOutputExample.Tokenizer
,SlidingWindowCheckMapper
,SocketClientSink
,StateBootstrapFunction
,StatefulSequenceSource
,StatefulStreamingJob.MySource
,StatefulStreamingJob.MyStatefulFunction
,StreamingFileSink
,TableFunctionCollector
,TableFunctionResultFuture
,TestListResultSink
,TwoPhaseCommitSinkFunction
,UpdatableTopNFunction
,WatermarkGenerator
,WindowReaderFunction
,WrappingCollector
,WrappingFunction
@Public public interface RichFunction extends Function
An base interface for all rich user-defined functions. This class defines methods for the life cycle of the functions, as well as methods to access the context in which the functions are executed.
-
-
Method Summary
All Methods Instance Methods Abstract Methods Modifier and Type Method Description void
close()
Tear-down method for the user code.IterationRuntimeContext
getIterationRuntimeContext()
Gets a specialized version of theRuntimeContext
, which has additional information about the iteration in which the function is executed.RuntimeContext
getRuntimeContext()
Gets the context that contains information about the UDF's runtime, such as the parallelism of the function, the subtask index of the function, or the name of the task that executes the function.void
open(OpenContext openContext)
Initialization method for the function.void
setRuntimeContext(RuntimeContext t)
Sets the function's runtime context.
-
-
-
Method Detail
-
open
@PublicEvolving void open(OpenContext openContext) throws Exception
Initialization method for the function. It is called before the actual working methods (like map or join) and thus suitable for one time setup work. For functions that are part of an iteration, this method will be invoked at the beginning of each iteration superstep.The openContext object passed to the function can be used for configuration and initialization. The openContext contains some necessary information that were configured on the function in the program composition.
public class MyFilter extends RichFilterFunction<String> { private String searchString; public void open(OpenContext openContext) { // initialize the value of searchString } public boolean filter(String value) { return value.equals(searchString); } }
- Parameters:
openContext
- The context containing information about the context in which the function is opened.- Throws:
Exception
- Implementations may forward exceptions, which are caught by the runtime. When the runtime catches an exception, it aborts the task and lets the fail-over logic decide whether to retry the task execution.
-
close
void close() throws Exception
Tear-down method for the user code. It is called after the last call to the main working methods (e.g. map or join). For functions that are part of an iteration, this method will be invoked after each iteration superstep.This method can be used for clean up work.
- Throws:
Exception
- Implementations may forward exceptions, which are caught by the runtime. When the runtime catches an exception, it aborts the task and lets the fail-over logic decide whether to retry the task execution.
-
getRuntimeContext
RuntimeContext getRuntimeContext()
Gets the context that contains information about the UDF's runtime, such as the parallelism of the function, the subtask index of the function, or the name of the task that executes the function.The RuntimeContext also gives access to the
Accumulator
s and theDistributedCache
.- Returns:
- The UDF's runtime context.
-
getIterationRuntimeContext
IterationRuntimeContext getIterationRuntimeContext()
Gets a specialized version of theRuntimeContext
, which has additional information about the iteration in which the function is executed. This IterationRuntimeContext is only available if the function is part of an iteration. Otherwise, this method throws an exception.- Returns:
- The IterationRuntimeContext.
- Throws:
IllegalStateException
- Thrown, if the function is not executed as part of an iteration.
-
setRuntimeContext
void setRuntimeContext(RuntimeContext t)
Sets the function's runtime context. Called by the framework when creating a parallel instance of the function.- Parameters:
t
- The runtime context.
-
-