K
- Type of the key by which state is keyed.public abstract class AbstractKeyedStateBackend<K> extends Object implements CheckpointableKeyedStateBackend<K>, CheckpointListener
SnapshotStrategy.snapshot(long, long, CheckpointStreamFactory, CheckpointOptions)
.KeyedStateBackend.KeySelectionListener<K>
Modifier and Type | Field and Description |
---|---|
protected CloseableRegistry |
cancelStreamRegistry
Registry for all opened streams, so they can be closed if the task using this backend is
closed.
|
protected InternalKeyContext<K> |
keyContext
The key context for this backend.
|
protected StreamCompressionDecorator |
keyGroupCompressionDecorator
Decorates the input and output streams to write key-groups compressed.
|
protected KeyGroupRange |
keyGroupRange
Range of key-groups for which this backend is responsible.
|
protected TypeSerializer<K> |
keySerializer
The key serializer.
|
protected TaskKvStateRegistry |
kvStateRegistry
KvStateRegistry helper for this task.
|
protected int |
numberOfKeyGroups
The number of key-groups aka max parallelism.
|
protected TtlTimeProvider |
ttlTimeProvider |
protected ClassLoader |
userCodeClassLoader |
Constructor and Description |
---|
AbstractKeyedStateBackend(TaskKvStateRegistry kvStateRegistry,
TypeSerializer<K> keySerializer,
ClassLoader userCodeClassLoader,
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
CloseableRegistry cancelStreamRegistry,
InternalKeyContext<K> keyContext) |
AbstractKeyedStateBackend(TaskKvStateRegistry kvStateRegistry,
TypeSerializer<K> keySerializer,
ClassLoader userCodeClassLoader,
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
CloseableRegistry cancelStreamRegistry,
StreamCompressionDecorator keyGroupCompressionDecorator,
InternalKeyContext<K> keyContext) |
Modifier and Type | Method and Description |
---|---|
<N,S extends State,T> |
applyToAllKeys(N namespace,
TypeSerializer<N> namespaceSerializer,
StateDescriptor<S,T> stateDescriptor,
KeyedStateFunction<K,S> function)
Applies the provided
KeyedStateFunction to the state with the provided StateDescriptor of all the currently active keys. |
void |
close() |
boolean |
deregisterKeySelectionListener(KeyedStateBackend.KeySelectionListener<K> listener)
Stop calling listener registered in
KeyedStateBackend.registerKeySelectionListener(org.apache.flink.runtime.state.KeyedStateBackend.KeySelectionListener<K>) . |
void |
dispose()
Closes the state backend, releasing all internal resources, but does not delete any
persistent checkpoint data.
|
K |
getCurrentKey() |
int |
getCurrentKeyGroupIndex() |
StreamCompressionDecorator |
getKeyGroupCompressionDecorator() |
KeyGroupRange |
getKeyGroupRange()
Returns the key groups which this state backend is responsible for.
|
TypeSerializer<K> |
getKeySerializer() |
int |
getNumberOfKeyGroups() |
<N,S extends State,V> |
getOrCreateKeyedState(TypeSerializer<N> namespaceSerializer,
StateDescriptor<S,V> stateDescriptor)
Creates or retrieves a keyed state backed by this state backend.
|
<N,S extends State> |
getPartitionedState(N namespace,
TypeSerializer<N> namespaceSerializer,
StateDescriptor<S,?> stateDescriptor)
TODO: NOTE: This method does a lot of work caching / retrieving states just to update the
namespace.
|
abstract int |
numKeyValueStateEntries()
Returns the total number of state entries across all keys/namespaces.
|
int |
numKeyValueStatesByName() |
void |
registerKeySelectionListener(KeyedStateBackend.KeySelectionListener<K> listener)
State backend will call
KeyedStateBackend.KeySelectionListener.keySelected(K) when key context is switched
if supported. |
boolean |
requiresLegacySynchronousTimerSnapshots() |
void |
setCurrentKey(K newKey)
Sets the current key that is used for partitioned state.
|
boolean |
supportsAsynchronousSnapshots() |
clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
getKeys, getKeysAndNamespaces
createInternalState, createInternalState
create
snapshot
notifyCheckpointAborted, notifyCheckpointComplete
protected final TypeSerializer<K> keySerializer
protected final int numberOfKeyGroups
protected final KeyGroupRange keyGroupRange
protected final TaskKvStateRegistry kvStateRegistry
protected CloseableRegistry cancelStreamRegistry
protected final ClassLoader userCodeClassLoader
protected final TtlTimeProvider ttlTimeProvider
protected final StreamCompressionDecorator keyGroupCompressionDecorator
protected final InternalKeyContext<K> keyContext
public AbstractKeyedStateBackend(TaskKvStateRegistry kvStateRegistry, TypeSerializer<K> keySerializer, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig, TtlTimeProvider ttlTimeProvider, CloseableRegistry cancelStreamRegistry, InternalKeyContext<K> keyContext)
public AbstractKeyedStateBackend(TaskKvStateRegistry kvStateRegistry, TypeSerializer<K> keySerializer, ClassLoader userCodeClassLoader, ExecutionConfig executionConfig, TtlTimeProvider ttlTimeProvider, CloseableRegistry cancelStreamRegistry, StreamCompressionDecorator keyGroupCompressionDecorator, InternalKeyContext<K> keyContext)
public void dispose()
dispose
in interface KeyedStateBackend<K>
dispose
in interface Disposable
public void setCurrentKey(K newKey)
KeyedStateBackend
setCurrentKey
in interface KeyedStateBackend<K>
newKey
- The new current key.KeyedStateBackend
public void registerKeySelectionListener(KeyedStateBackend.KeySelectionListener<K> listener)
KeyedStateBackend
KeyedStateBackend.KeySelectionListener.keySelected(K)
when key context is switched
if supported.registerKeySelectionListener
in interface KeyedStateBackend<K>
public boolean deregisterKeySelectionListener(KeyedStateBackend.KeySelectionListener<K> listener)
KeyedStateBackend
KeyedStateBackend.registerKeySelectionListener(org.apache.flink.runtime.state.KeyedStateBackend.KeySelectionListener<K>)
.deregisterKeySelectionListener
in interface KeyedStateBackend<K>
public TypeSerializer<K> getKeySerializer()
getKeySerializer
in interface KeyedStateBackend<K>
KeyedStateBackend
public K getCurrentKey()
getCurrentKey
in interface KeyedStateBackend<K>
KeyedStateBackend
public int getCurrentKeyGroupIndex()
KeyedStateBackend
public int getNumberOfKeyGroups()
KeyedStateBackend
public KeyGroupRange getKeyGroupRange()
CheckpointableKeyedStateBackend
getKeyGroupRange
in interface CheckpointableKeyedStateBackend<K>
KeyedStateBackend
public <N,S extends State,T> void applyToAllKeys(N namespace, TypeSerializer<N> namespaceSerializer, StateDescriptor<S,T> stateDescriptor, KeyedStateFunction<K,S> function) throws Exception
KeyedStateBackend
KeyedStateFunction
to the state with the provided StateDescriptor
of all the currently active keys.applyToAllKeys
in interface KeyedStateBackend<K>
N
- The type of the namespace.S
- The type of the state.namespace
- the namespace of the state.namespaceSerializer
- the serializer for the namespace.stateDescriptor
- the descriptor of the state to which the function is going to be
applied.function
- the function to be applied to the keyed state.Exception
KeyedStateBackend
public <N,S extends State,V> S getOrCreateKeyedState(TypeSerializer<N> namespaceSerializer, StateDescriptor<S,V> stateDescriptor) throws Exception
KeyedStateBackend
getOrCreateKeyedState
in interface KeyedStateBackend<K>
N
- The type of the namespace.S
- The type of the state.namespaceSerializer
- The serializer used for the namespace type of the statestateDescriptor
- The identifier for the state. This contains name and can create a
default state value.Exception
- Exceptions may occur during initialization of the state and should be
forwarded.KeyedStateBackend
public <N,S extends State> S getPartitionedState(N namespace, TypeSerializer<N> namespaceSerializer, StateDescriptor<S,?> stateDescriptor) throws Exception
getPartitionedState
in interface KeyedStateBackend<K>
N
- The type of the namespace.S
- The type of the state.stateDescriptor
- The identifier for the state. This contains name and can create a
default state value.Exception
- Exceptions may occur during initialization of the state and should be
forwarded.KeyedStateBackend
public void close() throws IOException
close
in interface Closeable
close
in interface AutoCloseable
IOException
@VisibleForTesting public boolean supportsAsynchronousSnapshots()
@VisibleForTesting public StreamCompressionDecorator getKeyGroupCompressionDecorator()
@VisibleForTesting public abstract int numKeyValueStateEntries()
@VisibleForTesting public int numKeyValueStatesByName()
public boolean requiresLegacySynchronousTimerSnapshots()
Copyright © 2014–2021 The Apache Software Foundation. All rights reserved.