Interface StateBackend
-
- All Superinterfaces:
Serializable
- All Known Subinterfaces:
ConfigurableStateBackend,DelegatingStateBackend
- All Known Implementing Classes:
AbstractFileStateBackend,AbstractManagedMemoryStateBackend,AbstractStateBackend,BatchExecutionStateBackend,HashMapStateBackend
@PublicEvolving public interface StateBackend extends Serializable
A State Backend defines how the state of a streaming application is stored locally within the cluster. Different State Backends store their state in different fashions, and use different data structures to hold the state of a running application.For example, the
hashmap state backendkeeps working state in the memory of the TaskManager. The backend is lightweight and without additional dependencies.The
EmbeddedRocksDBStateBackendstores working state in an embedded RocksDB and is able to scale working state to many terabytes in size, only limited by available disk space across all task managers.Raw Bytes Storage and Backends
The
StateBackendcreates services for keyed state and operator state.The
CheckpointableKeyedStateBackendandOperatorStateBackendcreated by this state backend define how to hold the working state for keys and operators. They also define how to checkpoint that state, frequently using the raw bytes storage (via theCheckpointStreamFactory). However, it is also possible that for example a keyed state backend simply implements the bridge to a key/value store, and that it does not need to store anything in the raw byte storage upon a checkpoint.Serializability
State Backends need to be
serializable, because they distributed across parallel processes (for distributed execution) together with the streaming application code.Because of that,
StateBackendimplementations (typically subclasses ofAbstractStateBackend) are meant to be like factories that create the proper states stores that provide access to the persistent storage and hold the keyed- and operator state data structures. That way, the State Backend can be very lightweight (contain only configurations) which makes it easier to be serializable.Thread Safety
State backend implementations have to be thread-safe. Multiple threads may be creating keyed-/operator state backends concurrently.
-
-
Nested Class Summary
Nested Classes Modifier and Type Interface Description static interfaceStateBackend.CustomInitializationMetricsstatic interfaceStateBackend.KeyedStateBackendParameters<K>Parameters passed tocreateKeyedStateBackend(KeyedStateBackendParameters).static interfaceStateBackend.OperatorStateBackendParametersParameters passed tocreateOperatorStateBackend(OperatorStateBackendParameters).
-
Method Summary
All Methods Instance Methods Abstract Methods Default Methods Modifier and Type Method Description default <K> AsyncKeyedStateBackend<K>createAsyncKeyedStateBackend(StateBackend.KeyedStateBackendParameters<K> parameters)Creates a newAsyncKeyedStateBackendwhich supports to access keyed state asynchronously.<K> CheckpointableKeyedStateBackend<K>createKeyedStateBackend(StateBackend.KeyedStateBackendParameters<K> parameters)Creates a newCheckpointableKeyedStateBackendthat is responsible for holding keyed state and checkpointing it.OperatorStateBackendcreateOperatorStateBackend(StateBackend.OperatorStateBackendParameters parameters)Creates a newOperatorStateBackendthat can be used for storing operator state.default StringgetName()Return the name of this backend, default is simple class name.default booleansupportsAsyncKeyedStateBackend()Tells if a state backend supports theAsyncKeyedStateBackend.default booleansupportsNoClaimRestoreMode()Tells if a state backend supports theRecoveryClaimMode.NO_CLAIMmode.default booleansupportsSavepointFormat(org.apache.flink.core.execution.SavepointFormatType formatType)default booleanuseManagedMemory()Whether the state backend uses Flink's managed memory.
-
-
-
Method Detail
-
getName
default String getName()
Return the name of this backend, default is simple class name.DelegatingStateBackendmay return the simple class name of the delegated backend.
-
createKeyedStateBackend
<K> CheckpointableKeyedStateBackend<K> createKeyedStateBackend(StateBackend.KeyedStateBackendParameters<K> parameters) throws Exception
Creates a newCheckpointableKeyedStateBackendthat is responsible for holding keyed state and checkpointing it.Keyed State is state where each value is bound to a key.
- Type Parameters:
K- The type of the keys by which the state is organized.- Parameters:
parameters- The arguments bundle for creatingCheckpointableKeyedStateBackend.- Returns:
- The Keyed State Backend for the given job, operator, and key group range.
- Throws:
Exception- This method may forward all exceptions that occur while instantiating the backend.
-
createAsyncKeyedStateBackend
@Experimental default <K> AsyncKeyedStateBackend<K> createAsyncKeyedStateBackend(StateBackend.KeyedStateBackendParameters<K> parameters) throws Exception
Creates a newAsyncKeyedStateBackendwhich supports to access keyed state asynchronously.Keyed State is state where each value is bound to a key.
- Type Parameters:
K- The type of the keys by which the state is organized.- Parameters:
parameters- The arguments bundle for creatingAsyncKeyedStateBackend.- Returns:
- The Async Keyed State Backend for the given job, operator.
- Throws:
Exception- This method may forward all exceptions that occur while instantiating the backend.
-
supportsAsyncKeyedStateBackend
@Experimental default boolean supportsAsyncKeyedStateBackend()
Tells if a state backend supports theAsyncKeyedStateBackend.If a state backend supports
AsyncKeyedStateBackend, it could usecreateAsyncKeyedStateBackend(KeyedStateBackendParameters)to create an async keyed state backend to access keyed state asynchronously.- Returns:
- If the state backend supports
AsyncKeyedStateBackend.
-
createOperatorStateBackend
OperatorStateBackend createOperatorStateBackend(StateBackend.OperatorStateBackendParameters parameters) throws Exception
Creates a newOperatorStateBackendthat can be used for storing operator state.Operator state is state that is associated with parallel operator (or function) instances, rather than with keys.
- Parameters:
parameters- The arguments bundle for creatingOperatorStateBackend.- Returns:
- The OperatorStateBackend for operator identified by the job and operator identifier.
- Throws:
Exception- This method may forward all exceptions that occur while instantiating the backend.
-
useManagedMemory
default boolean useManagedMemory()
Whether the state backend uses Flink's managed memory.
-
supportsNoClaimRestoreMode
default boolean supportsNoClaimRestoreMode()
Tells if a state backend supports theRecoveryClaimMode.NO_CLAIMmode.If a state backend supports
NO_CLAIMmode, it should create an independent snapshot when it receivesCheckpointType.FULL_CHECKPOINTinSnapshotable.snapshot(long, long, CheckpointStreamFactory, CheckpointOptions).- Returns:
- If the state backend supports
RecoveryClaimMode.NO_CLAIMmode.
-
supportsSavepointFormat
default boolean supportsSavepointFormat(org.apache.flink.core.execution.SavepointFormatType formatType)
-
-