Interface RuntimeContext
-
- All Known Subinterfaces:
IterationRuntimeContext
- All Known Implementing Classes:
AbstractRuntimeUDFContext,RuntimeUDFContext
@Public public interface RuntimeContextA RuntimeContext contains information about the context in which functions are executed. Each parallel instance of the function will have a context through which it can access static contextual information (such as the current parallelism) and other constructs like accumulators and broadcast variables.A function can, during runtime, obtain the RuntimeContext via a call to
AbstractRichFunction.getRuntimeContext().
-
-
Method Summary
All Methods Instance Methods Abstract Methods Modifier and Type Method Description <V,A extends Serializable>
voidaddAccumulator(String name, Accumulator<V,A> accumulator)Add this accumulator.<V,A extends Serializable>
Accumulator<V,A>getAccumulator(String name)Get an existing accumulator object.<IN,ACC,OUT>
AggregatingState<IN,OUT>getAggregatingState(AggregatingStateDescriptor<IN,ACC,OUT> stateProperties)Gets a handle to the system's key/value aggregating state.intgetAttemptNumber()Gets the attempt number of this parallel subtask.<RT> List<RT>getBroadcastVariable(String name)Returns the result bound to the broadcast variable identified by the givenname.<T,C>
CgetBroadcastVariableWithInitializer(String name, BroadcastVariableInitializer<T,C> initializer)Returns the result bound to the broadcast variable identified by the givenname.DistributedCachegetDistributedCache()Returns theDistributedCacheto get the local temporary file copies of files otherwise not locally accessible.DoubleCountergetDoubleCounter(String name)Convenience function to create a counter object for doubles.ExecutionConfiggetExecutionConfig()Returns theExecutionConfigfor the currently executing job.Set<ExternalResourceInfo>getExternalResourceInfos(String resourceName)Get the specific external resource information by the resourceName.HistogramgetHistogram(String name)Convenience function to create a counter object for histograms.intgetIndexOfThisSubtask()Gets the number of this parallel subtask.IntCountergetIntCounter(String name)Convenience function to create a counter object for integers.JobIDgetJobId()The ID of the current job.<T> ListState<T>getListState(ListStateDescriptor<T> stateProperties)Gets a handle to the system's key/value list state.LongCountergetLongCounter(String name)Convenience function to create a counter object for longs.<UK,UV>
MapState<UK,UV>getMapState(MapStateDescriptor<UK,UV> stateProperties)Gets a handle to the system's key/value map state.intgetMaxNumberOfParallelSubtasks()Gets the number of max-parallelism with which the parallel task runs.org.apache.flink.metrics.groups.OperatorMetricGroupgetMetricGroup()Returns the metric group for this parallel subtask.intgetNumberOfParallelSubtasks()Gets the parallelism with which the parallel task runs.<T> ReducingState<T>getReducingState(ReducingStateDescriptor<T> stateProperties)Gets a handle to the system's key/value reducing state.<T> ValueState<T>getState(ValueStateDescriptor<T> stateProperties)Gets a handle to the system's key/value state.StringgetTaskName()Returns the name of the task in which the UDF runs, as assigned during plan construction.StringgetTaskNameWithSubtasks()Returns the name of the task, appended with the subtask indicator, such as "MyTask (3/6)#1", where 3 would be (getIndexOfThisSubtask()+ 1), and 6 would begetNumberOfParallelSubtasks(), and 1 would begetAttemptNumber().ClassLoadergetUserCodeClassLoader()Gets the ClassLoader to load classes that are not in system's classpath, but are part of the jar file of a user job.booleanhasBroadcastVariable(String name)Tests for the existence of the broadcast variable identified by the givenname.voidregisterUserCodeClassLoaderReleaseHookIfAbsent(String releaseHookName, Runnable releaseHook)Registers a custom hook for the user code class loader release.
-
-
-
Method Detail
-
getJobId
JobID getJobId()
The ID of the current job. Note that Job ID can change in particular upon manual restart. The returned ID should NOT be used for any job management tasks.
-
getTaskName
String getTaskName()
Returns the name of the task in which the UDF runs, as assigned during plan construction.- Returns:
- The name of the task in which the UDF runs.
-
getMetricGroup
@PublicEvolving org.apache.flink.metrics.groups.OperatorMetricGroup getMetricGroup()
Returns the metric group for this parallel subtask.- Returns:
- The metric group for this parallel subtask.
-
getNumberOfParallelSubtasks
int getNumberOfParallelSubtasks()
Gets the parallelism with which the parallel task runs.- Returns:
- The parallelism with which the parallel task runs.
-
getMaxNumberOfParallelSubtasks
@PublicEvolving int getMaxNumberOfParallelSubtasks()
Gets the number of max-parallelism with which the parallel task runs.- Returns:
- The max-parallelism with which the parallel task runs.
-
getIndexOfThisSubtask
int getIndexOfThisSubtask()
Gets the number of this parallel subtask. The numbering starts from 0 and goes up to parallelism-1 (parallelism as returned bygetNumberOfParallelSubtasks()).- Returns:
- The index of the parallel subtask.
-
getAttemptNumber
int getAttemptNumber()
Gets the attempt number of this parallel subtask. First attempt is numbered 0.- Returns:
- Attempt number of the subtask.
-
getTaskNameWithSubtasks
String getTaskNameWithSubtasks()
Returns the name of the task, appended with the subtask indicator, such as "MyTask (3/6)#1", where 3 would be (getIndexOfThisSubtask()+ 1), and 6 would begetNumberOfParallelSubtasks(), and 1 would begetAttemptNumber().- Returns:
- The name of the task, with subtask indicator.
-
getExecutionConfig
ExecutionConfig getExecutionConfig()
Returns theExecutionConfigfor the currently executing job.
-
getUserCodeClassLoader
ClassLoader getUserCodeClassLoader()
Gets the ClassLoader to load classes that are not in system's classpath, but are part of the jar file of a user job.- Returns:
- The ClassLoader for user code classes.
-
registerUserCodeClassLoaderReleaseHookIfAbsent
@PublicEvolving void registerUserCodeClassLoaderReleaseHookIfAbsent(String releaseHookName, Runnable releaseHook)
Registers a custom hook for the user code class loader release.The release hook is executed just before the user code class loader is being released. Registration only happens if no hook has been registered under this name already.
- Parameters:
releaseHookName- name of the release hook.releaseHook- release hook which is executed just before the user code class loader is being released
-
addAccumulator
<V,A extends Serializable> void addAccumulator(String name, Accumulator<V,A> accumulator)
Add this accumulator. Throws an exception if the accumulator already exists in the same Task. Note that the Accumulator name must have an unique name across the Flink job. Otherwise you will get an error when incompatible accumulators from different Tasks are combined at the JobManager upon job completion.
-
getAccumulator
<V,A extends Serializable> Accumulator<V,A> getAccumulator(String name)
Get an existing accumulator object. The accumulator must have been added previously in this local runtime context.Throws an exception if the accumulator does not exist or if the accumulator exists, but with different type.
-
getIntCounter
@PublicEvolving IntCounter getIntCounter(String name)
Convenience function to create a counter object for integers.
-
getLongCounter
@PublicEvolving LongCounter getLongCounter(String name)
Convenience function to create a counter object for longs.
-
getDoubleCounter
@PublicEvolving DoubleCounter getDoubleCounter(String name)
Convenience function to create a counter object for doubles.
-
getHistogram
@PublicEvolving Histogram getHistogram(String name)
Convenience function to create a counter object for histograms.
-
getExternalResourceInfos
@PublicEvolving Set<ExternalResourceInfo> getExternalResourceInfos(String resourceName)
Get the specific external resource information by the resourceName.- Parameters:
resourceName- of the required external resource- Returns:
- information set of the external resource identified by the resourceName
-
hasBroadcastVariable
@PublicEvolving boolean hasBroadcastVariable(String name)
Tests for the existence of the broadcast variable identified by the givenname.- Parameters:
name- The name under which the broadcast variable is registered;- Returns:
- Whether a broadcast variable exists for the given name.
-
getBroadcastVariable
<RT> List<RT> getBroadcastVariable(String name)
Returns the result bound to the broadcast variable identified by the givenname.IMPORTANT: The broadcast variable data structure is shared between the parallel tasks on one machine. Any access that modifies its internal state needs to be manually synchronized by the caller.
- Parameters:
name- The name under which the broadcast variable is registered;- Returns:
- The broadcast variable, materialized as a list of elements.
-
getBroadcastVariableWithInitializer
<T,C> C getBroadcastVariableWithInitializer(String name, BroadcastVariableInitializer<T,C> initializer)
Returns the result bound to the broadcast variable identified by the givenname. The broadcast variable is returned as a shared data structure that is initialized with the givenBroadcastVariableInitializer.IMPORTANT: The broadcast variable data structure is shared between the parallel tasks on one machine. Any access that modifies its internal state needs to be manually synchronized by the caller.
- Parameters:
name- The name under which the broadcast variable is registered;initializer- The initializer that creates the shared data structure of the broadcast variable from the sequence of elements.- Returns:
- The broadcast variable, materialized as a list of elements.
-
getDistributedCache
DistributedCache getDistributedCache()
Returns theDistributedCacheto get the local temporary file copies of files otherwise not locally accessible.- Returns:
- The distributed cache of the worker executing this instance.
-
getState
@PublicEvolving <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties)
Gets a handle to the system's key/value state. The key/value state is only accessible if the function is executed on a KeyedStream. On each access, the state exposes the value for the key of the element currently processed by the function. Each function may have multiple partitioned states, addressed with different names.Because the scope of each value is the key of the currently processed element, and the elements are distributed by the Flink runtime, the system can transparently scale out and redistribute the state and KeyedStream.
The following code example shows how to implement a continuous counter that counts how many times elements of a certain key occur, and emits an updated count for that element on each occurrence.
DataStream<MyType> stream = ...; KeyedStream<MyType> keyedStream = stream.keyBy("id"); keyedStream.map(new RichMapFunction<MyType, Tuple2<MyType, Long>>() { private ValueState<Long> state; public void open(Configuration cfg) { state = getRuntimeContext().getState( new ValueStateDescriptor<Long>("count", LongSerializer.INSTANCE, 0L)); } public Tuple2<MyType, Long> map(MyType value) { long count = state.value() + 1; state.update(count); return new Tuple2<>(value, count); } });- Type Parameters:
T- The type of value stored in the state.- Parameters:
stateProperties- The descriptor defining the properties of the stats.- Returns:
- The partitioned state object.
- Throws:
UnsupportedOperationException- Thrown, if no partitioned state is available for the function (function is not part of a KeyedStream).
-
getListState
@PublicEvolving <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties)
Gets a handle to the system's key/value list state. This state is similar to the state accessed viagetState(ValueStateDescriptor), but is optimized for state that holds lists. One can add elements to the list, or retrieve the list as a whole.This state is only accessible if the function is executed on a KeyedStream.
DataStream<MyType> stream = ...; KeyedStream<MyType> keyedStream = stream.keyBy("id"); keyedStream.map(new RichFlatMapFunction<MyType, List<MyType>>() { private ListState<MyType> state; public void open(Configuration cfg) { state = getRuntimeContext().getListState( new ListStateDescriptor<>("myState", MyType.class)); } public void flatMap(MyType value, Collector<MyType> out) { if (value.isDivider()) { for (MyType t : state.get()) { out.collect(t); } } else { state.add(value); } } });- Type Parameters:
T- The type of value stored in the state.- Parameters:
stateProperties- The descriptor defining the properties of the stats.- Returns:
- The partitioned state object.
- Throws:
UnsupportedOperationException- Thrown, if no partitioned state is available for the function (function is not part os a KeyedStream).
-
getReducingState
@PublicEvolving <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties)
Gets a handle to the system's key/value reducing state. This state is similar to the state accessed viagetState(ValueStateDescriptor), but is optimized for state that aggregates values.This state is only accessible if the function is executed on a KeyedStream.
DataStream<MyType> stream = ...; KeyedStream<MyType> keyedStream = stream.keyBy("id"); keyedStream.map(new RichMapFunction<MyType, List<MyType>>() { private ReducingState<Long> state; public void open(Configuration cfg) { state = getRuntimeContext().getReducingState( new ReducingStateDescriptor<>("sum", (a, b) -> a + b, Long.class)); } public Tuple2<MyType, Long> map(MyType value) { state.add(value.count()); return new Tuple2<>(value, state.get()); } });- Type Parameters:
T- The type of value stored in the state.- Parameters:
stateProperties- The descriptor defining the properties of the stats.- Returns:
- The partitioned state object.
- Throws:
UnsupportedOperationException- Thrown, if no partitioned state is available for the function (function is not part of a KeyedStream).
-
getAggregatingState
@PublicEvolving <IN,ACC,OUT> AggregatingState<IN,OUT> getAggregatingState(AggregatingStateDescriptor<IN,ACC,OUT> stateProperties)
Gets a handle to the system's key/value aggregating state. This state is similar to the state accessed viagetState(ValueStateDescriptor), but is optimized for state that aggregates values with different types.This state is only accessible if the function is executed on a KeyedStream.
DataStream<MyType> stream = ...; KeyedStream<MyType> keyedStream = stream.keyBy("id"); AggregateFunction<...> aggregateFunction = ... keyedStream.map(new RichMapFunction<MyType, List<MyType>>() { private AggregatingState<MyType, Long> state; public void open(Configuration cfg) { state = getRuntimeContext().getAggregatingState( new AggregatingStateDescriptor<>("sum", aggregateFunction, Long.class)); } public Tuple2<MyType, Long> map(MyType value) { state.add(value); return new Tuple2<>(value, state.get()); } });- Type Parameters:
IN- The type of the values that are added to the state.ACC- The type of the accumulator (intermediate aggregation state).OUT- The type of the values that are returned from the state.- Parameters:
stateProperties- The descriptor defining the properties of the stats.- Returns:
- The partitioned state object.
- Throws:
UnsupportedOperationException- Thrown, if no partitioned state is available for the function (function is not part of a KeyedStream).
-
getMapState
@PublicEvolving <UK,UV> MapState<UK,UV> getMapState(MapStateDescriptor<UK,UV> stateProperties)
Gets a handle to the system's key/value map state. This state is similar to the state accessed viagetState(ValueStateDescriptor), but is optimized for state that is composed of user-defined key-value pairsThis state is only accessible if the function is executed on a KeyedStream.
DataStream<MyType> stream = ...; KeyedStream<MyType> keyedStream = stream.keyBy("id"); keyedStream.map(new RichMapFunction<MyType, List<MyType>>() { private MapState<MyType, Long> state; public void open(Configuration cfg) { state = getRuntimeContext().getMapState( new MapStateDescriptor<>("sum", MyType.class, Long.class)); } public Tuple2<MyType, Long> map(MyType value) { return new Tuple2<>(value, state.get(value)); } });- Type Parameters:
UK- The type of the user keys stored in the state.UV- The type of the user values stored in the state.- Parameters:
stateProperties- The descriptor defining the properties of the stats.- Returns:
- The partitioned state object.
- Throws:
UnsupportedOperationException- Thrown, if no partitioned state is available for the function (function is not part of a KeyedStream).
-
-