@Public public interface RuntimeContext
A function can, during runtime, obtain the RuntimeContext via a call to
AbstractRichFunction.getRuntimeContext()
.
Modifier and Type | Method and Description |
---|---|
<V,A extends Serializable> |
addAccumulator(String name,
Accumulator<V,A> accumulator)
Add this accumulator.
|
<V,A extends Serializable> |
getAccumulator(String name)
Get an existing accumulator object.
|
Map<String,Accumulator<?,?>> |
getAllAccumulators()
Deprecated.
Use getAccumulator(..) to obtain the value of an accumulator.
|
int |
getAttemptNumber()
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
given
name . |
<T,C> C |
getBroadcastVariableWithInitializer(String name,
BroadcastVariableInitializer<T,C> initializer)
Returns the result bound to the broadcast variable identified by the
given
name . |
DistributedCache |
getDistributedCache()
Returns the
DistributedCache to get the local temporary file copies of files otherwise not
locally accessible. |
DoubleCounter |
getDoubleCounter(String name)
Convenience function to create a counter object for doubles.
|
ExecutionConfig |
getExecutionConfig()
Returns the
ExecutionConfig for the currently executing
job. |
Histogram |
getHistogram(String name)
Convenience function to create a counter object for histograms.
|
int |
getIndexOfThisSubtask()
Gets the number of this parallel subtask.
|
IntCounter |
getIntCounter(String name)
Convenience function to create a counter object for integers.
|
<S> OperatorState<S> |
getKeyValueState(String name,
Class<S> stateType,
S defaultState)
Deprecated.
Use the more expressive
getState(ValueStateDescriptor) instead. |
<S> OperatorState<S> |
getKeyValueState(String name,
TypeInformation<S> stateType,
S defaultState)
Deprecated.
Use the more expressive
getState(ValueStateDescriptor) instead. |
<T> ListState<T> |
getListState(ListStateDescriptor<T> stateProperties)
Gets a handle to the system's key/value list state.
|
LongCounter |
getLongCounter(String name)
Convenience function to create a counter object for longs.
|
int |
getNumberOfParallelSubtasks()
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 list state.
|
<T> ValueState<T> |
getState(ValueStateDescriptor<T> stateProperties)
Gets a handle to the system's key/value state.
|
String |
getTaskName()
Returns the name of the task in which the UDF runs, as assigned during plan construction.
|
String |
getTaskNameWithSubtasks()
Returns the name of the task, appended with the subtask indicator, such as "MyTask (3/6)",
where 3 would be (
getIndexOfThisSubtask() + 1), and 6 would be
getNumberOfParallelSubtasks() . |
ClassLoader |
getUserCodeClassLoader()
Gets the ClassLoader to load classes that were are not in system's classpath, but are part of the
jar file of a user job.
|
String getTaskName()
int getNumberOfParallelSubtasks()
int getIndexOfThisSubtask()
getNumberOfParallelSubtasks()
).int getAttemptNumber()
String getTaskNameWithSubtasks()
getIndexOfThisSubtask()
+ 1), and 6 would be
getNumberOfParallelSubtasks()
.ExecutionConfig getExecutionConfig()
ExecutionConfig
for the currently executing
job.ClassLoader getUserCodeClassLoader()
<V,A extends Serializable> void addAccumulator(String name, Accumulator<V,A> accumulator)
<V,A extends Serializable> Accumulator<V,A> getAccumulator(String name)
@Deprecated @PublicEvolving Map<String,Accumulator<?,?>> getAllAccumulators()
@PublicEvolving IntCounter getIntCounter(String name)
@PublicEvolving LongCounter getLongCounter(String name)
@PublicEvolving DoubleCounter getDoubleCounter(String name)
@PublicEvolving Histogram getHistogram(String name)
<RT> List<RT> getBroadcastVariable(String name)
name
.
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.
name
- The name under which the broadcast variable is registered;<T,C> C getBroadcastVariableWithInitializer(String name, BroadcastVariableInitializer<T,C> initializer)
name
. The broadcast variable is returned as a shared data structure
that is initialized with the given BroadcastVariableInitializer
.
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.
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.DistributedCache getDistributedCache()
DistributedCache
to get the local temporary file copies of files otherwise not
locally accessible.@PublicEvolving <T> ValueState<T> getState(ValueStateDescriptor<T> stateProperties)
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> count;
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(value);
return new Tuple2<>(value, count);
}
});
T
- The type of value stored in the state.stateProperties
- The descriptor defining the properties of the stats.UnsupportedOperationException
- Thrown, if no partitioned state is available for the
function (function is not part of a KeyedStream).@PublicEvolving <T> ListState<T> getListState(ListStateDescriptor<T> stateProperties)
getState(ValueStateDescriptor)
, but is optimized for state that
holds lists. One can adds 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);
}
}
});
T
- The type of value stored in the state.stateProperties
- The descriptor defining the properties of the stats.UnsupportedOperationException
- Thrown, if no partitioned state is available for the
function (function is not part os a KeyedStream).@PublicEvolving <T> ReducingState<T> getReducingState(ReducingStateDescriptor<T> stateProperties)
getState(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> sum;
public void open(Configuration cfg) {
state = getRuntimeContext().getReducingState(
new ReducingStateDescriptor<>("sum", MyType.class, 0L, (a, b) -> a + b));
}
public Tuple2<MyType, Long> map(MyType value) {
sum.add(value.count());
return new Tuple2<>(value, sum.get());
}
});
T
- The type of value stored in the state.stateProperties
- The descriptor defining the properties of the stats.UnsupportedOperationException
- Thrown, if no partitioned state is available for the
function (function is not part of a KeyedStream).@Deprecated @PublicEvolving <S> OperatorState<S> getKeyValueState(String name, Class<S> stateType, S defaultState)
getState(ValueStateDescriptor)
instead.ValueState.value()
, the key/value state will
return the value bound to the key of the element currently processed by the function.
Each operator may maintain multiple key/value 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 State<Long> state;
public void open(Configuration cfg) {
state = getRuntimeContext().getKeyValueState(Long.class, 0L);
}
public Tuple2<MyType, Long> map(MyType value) {
long count = state.value();
state.update(value + 1);
return new Tuple2<>(value, count);
}
});
This method attempts to deduce the type information from the given type class. If the
full type cannot be determined from the class (for example because of generic parameters),
the TypeInformation object must be manually passed via
getKeyValueState(String, TypeInformation, Object)
.
S
- The type of the state.name
- The name of the key/value state.stateType
- The class of the type that is stored in the state. Used to generate
serializers for managed memory and checkpointing.defaultState
- The default state value, returned when the state is accessed and
no value has yet been set for the key. May be null.UnsupportedOperationException
- Thrown, if no key/value state is available for the
function (function is not part os a KeyedStream).@Deprecated @PublicEvolving <S> OperatorState<S> getKeyValueState(String name, TypeInformation<S> stateType, S defaultState)
getState(ValueStateDescriptor)
instead.ValueState.value()
, the key/value state will
return the value bound to the key of the element currently processed by the function.
Each operator may maintain multiple key/value 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 State<Long> state;
public void open(Configuration cfg) {
state = getRuntimeContext().getKeyValueState(Long.class, 0L);
}
public Tuple2<MyType, Long> map(MyType value) {
long count = state.value();
state.update(value + 1);
return new Tuple2<>(value, count);
}
});
S
- The type of the state.name
- The name of the key/value state.stateType
- The type information for the type that is stored in the state.
Used to create serializers for managed memory and checkpoints.defaultState
- The default state value, returned when the state is accessed and
no value has yet been set for the key. May be null.UnsupportedOperationException
- Thrown, if no key/value state is available for the
function (function is not part os a KeyedStream).Copyright © 2014–2017 The Apache Software Foundation. All rights reserved.