K
- Type of the key by which state is keyed.public abstract class AbstractKeyedStateBackend<K> extends Object implements KeyedStateBackend<K>, SnapshotStrategy<SnapshotResult<KeyedStateHandle>>, Closeable, 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 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 StateSerializerProvider<K> |
keySerializerProvider
StateSerializerProvider for our 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,
StateSerializerProvider<K> keySerializerProvider,
ClassLoader userCodeClassLoader,
int numberOfKeyGroups,
KeyGroupRange keyGroupRange,
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
CloseableRegistry cancelStreamRegistry,
StreamCompressionDecorator keyGroupCompressionDecorator) |
AbstractKeyedStateBackend(TaskKvStateRegistry kvStateRegistry,
TypeSerializer<K> keySerializer,
ClassLoader userCodeClassLoader,
int numberOfKeyGroups,
KeyGroupRange keyGroupRange,
ExecutionConfig executionConfig,
TtlTimeProvider ttlTimeProvider,
CloseableRegistry cancelStreamRegistry) |
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()
Used by states to access the current key.
|
int |
getCurrentKeyGroupIndex()
Returns the key-group to which the current key belongs.
|
StreamCompressionDecorator |
getKeyGroupCompressionDecorator() |
KeyGroupRange |
getKeyGroupRange()
Returns the key groups for this backend.
|
TypeSerializer<K> |
getKeySerializer()
TypeSerializer for the state backend key type. |
int |
getNumberOfKeyGroups()
Returns the number of key-groups aka max parallelism.
|
<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.
|
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
createInternalState, createInternalState
create
snapshot
notifyCheckpointComplete
protected final StateSerializerProvider<K> keySerializerProvider
StateSerializerProvider
for our key serializer.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
public AbstractKeyedStateBackend(TaskKvStateRegistry kvStateRegistry, TypeSerializer<K> keySerializer, ClassLoader userCodeClassLoader, int numberOfKeyGroups, KeyGroupRange keyGroupRange, ExecutionConfig executionConfig, TtlTimeProvider ttlTimeProvider, CloseableRegistry cancelStreamRegistry)
public AbstractKeyedStateBackend(TaskKvStateRegistry kvStateRegistry, StateSerializerProvider<K> keySerializerProvider, ClassLoader userCodeClassLoader, int numberOfKeyGroups, KeyGroupRange keyGroupRange, ExecutionConfig executionConfig, TtlTimeProvider ttlTimeProvider, CloseableRegistry cancelStreamRegistry, StreamCompressionDecorator keyGroupCompressionDecorator)
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()
InternalKeyContext
TypeSerializer
for the state backend key type.getKeySerializer
in interface InternalKeyContext<K>
KeyedStateBackend
public K getCurrentKey()
InternalKeyContext
getCurrentKey
in interface InternalKeyContext<K>
KeyedStateBackend
public int getCurrentKeyGroupIndex()
InternalKeyContext
getCurrentKeyGroupIndex
in interface InternalKeyContext<K>
KeyedStateBackend
public int getNumberOfKeyGroups()
InternalKeyContext
getNumberOfKeyGroups
in interface InternalKeyContext<K>
KeyedStateBackend
public KeyGroupRange getKeyGroupRange()
InternalKeyContext
getKeyGroupRange
in interface InternalKeyContext<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()
public boolean requiresLegacySynchronousTimerSnapshots()
Copyright © 2014–2020 The Apache Software Foundation. All rights reserved.