Interface KeyedStateHandle

All Superinterfaces:
CompositeStateHandle, Serializable, StateObject
All Known Subinterfaces:
ChangelogStateBackendHandle, ChangelogStateHandle, CheckpointBoundKeyedStateHandle, IncrementalKeyedStateHandle, SavepointKeyedStateHandle
All Known Implementing Classes:
AbstractIncrementalStateHandle, ChangelogStateBackendHandle.ChangelogStateBackendHandleImpl, ChangelogStateBackendLocalHandle, ChangelogStateHandleStreamImpl, IncrementalLocalKeyedStateHandle, IncrementalRemoteKeyedStateHandle, InMemoryChangelogStateHandle, KeyGroupsSavepointStateHandle, KeyGroupsStateHandle

public interface KeyedStateHandle extends CompositeStateHandle
Base for the handles of the checkpointed states in keyed streams. When recovering from failures, the handle will be passed to all tasks whose key group ranges overlap with it.
  • Method Details

    • getKeyGroupRange

      KeyGroupRange getKeyGroupRange()
      Returns the range of the key groups contained in the state.
    • getIntersection

      @Nullable KeyedStateHandle getIntersection(KeyGroupRange keyGroupRange)
      Returns a state over a range that is the intersection between this handle's key-group range and the provided key-group range.
      Parameters:
      keyGroupRange - The key group range to intersect with, will return null if the intersection of this handle's key-group and the provided key-group is empty.
    • getStateHandleId

      StateHandleID getStateHandleId()
      Returns a unique state handle id to distinguish with other keyed state handles.

      If this state handle would be used as materialized part of ChangelogStateBackendHandle, it should persist this state handle id when serializing of checkpoint and deserialize it back to ensure the state handle id is constant.

      Returns:
      A unique state handle id.