@PublicEvolving public interface OperatorStateStore
Modifier and Type | Method and Description |
---|---|
<K,V> BroadcastState<K,V> |
getBroadcastState(MapStateDescriptor<K,V> stateDescriptor)
Creates (or restores) a
broadcast state . |
<S> ListState<S> |
getListState(ListStateDescriptor<S> stateDescriptor)
Creates (or restores) a list state.
|
<S> ListState<S> |
getOperatorState(ListStateDescriptor<S> stateDescriptor)
Deprecated.
since 1.3.0. This was deprecated as part of a refinement to the function names.
Please use
getListState(ListStateDescriptor) instead. |
Set<String> |
getRegisteredBroadcastStateNames()
Returns a set with the names of all currently registered broadcast states.
|
Set<String> |
getRegisteredStateNames()
Returns a set with the names of all currently registered states.
|
<T extends Serializable> |
getSerializableListState(String stateName)
Deprecated.
since 1.3.0. Using Java serialization for persisting state is not encouraged.
Please use
getListState(ListStateDescriptor) instead. |
<S> ListState<S> |
getUnionListState(ListStateDescriptor<S> stateDescriptor)
Creates (or restores) a list state.
|
<K,V> BroadcastState<K,V> getBroadcastState(MapStateDescriptor<K,V> stateDescriptor) throws Exception
broadcast state
. This type of state can only be created to store
the state of a BroadcastStream
. Each state is registered under a unique name.
The provided serializer is used to de/serialize the state in case of checkpointing (snapshot/restore).
The returned broadcast state has key-value
format.
CAUTION: the user has to guarantee that all task instances store the same elements in this type of state.
Each operator instance individually maintains and stores elements in the broadcast state. The fact that the incoming stream is a broadcast one guarantees that all instances see all the elements. Upon recovery or re-scaling, the same state is given to each of the instances. To avoid hotspots, each task reads its previous partition, and if there are more tasks (scale up), then the new instances read from the old instances in a round robin fashion. This is why each instance has to guarantee that it stores the same elements as the rest. If not, upon recovery or rescaling you may have unpredictable redistribution of the partitions, thus unpredictable results.
K
- The type of the keys in the broadcast state.V
- The type of the values in the broadcast state.stateDescriptor
- The descriptor for this state, providing a name, a serializer for the keys and one for the
values.Exception
<S> ListState<S> getListState(ListStateDescriptor<S> stateDescriptor) throws Exception
Note the semantic differences between an operator list state and a keyed list state
(see KeyedStateStore.getListState(ListStateDescriptor)
). Under the context of operator state,
the list is a collection of state items that are independent from each other and eligible for redistribution
across operator instances in case of changed operator parallelism. In other words, these state items are
the finest granularity at which non-keyed state can be redistributed, and should not be correlated with
each other.
The redistribution scheme of this list state upon operator rescaling is a round-robin pattern, such that the logical whole state (a concatenation of all the lists of state elements previously managed by each operator before the restore) is evenly divided into as many sublists as there are parallel operators.
S
- The generic type of the statestateDescriptor
- The descriptor for this state, providing a name and serializer.Exception
<S> ListState<S> getUnionListState(ListStateDescriptor<S> stateDescriptor) throws Exception
Note the semantic differences between an operator list state and a keyed list state
(see KeyedStateStore.getListState(ListStateDescriptor)
). Under the context of operator state,
the list is a collection of state items that are independent from each other and eligible for redistribution
across operator instances in case of changed operator parallelism. In other words, these state items are
the finest granularity at which non-keyed state can be redistributed, and should not be correlated with
each other.
The redistribution scheme of this list state upon operator rescaling is a broadcast pattern, such that the logical whole state (a concatenation of all the lists of state elements previously managed by each operator before the restore) is restored to all parallel operators so that each of them will get the union of all state items before the restore.
S
- The generic type of the statestateDescriptor
- The descriptor for this state, providing a name and serializer.Exception
Set<String> getRegisteredStateNames()
Set<String> getRegisteredBroadcastStateNames()
@Deprecated <S> ListState<S> getOperatorState(ListStateDescriptor<S> stateDescriptor) throws Exception
getListState(ListStateDescriptor)
instead.The items in the list are repartitionable by the system in case of changed operator parallelism.
S
- The generic type of the statestateDescriptor
- The descriptor for this state, providing a name and serializer.Exception
@Deprecated <T extends Serializable> ListState<T> getSerializableListState(String stateName) throws Exception
getListState(ListStateDescriptor)
instead.This is a simple convenience method. For more flexibility on how state serialization
should happen, use the getListState(ListStateDescriptor)
method.
stateName
- The name of state to createException
Copyright © 2014–2019 The Apache Software Foundation. All rights reserved.