Interface StateStore
- All Known Subinterfaces:
- KeyValueStore<K,,- V> - SessionStore<K,,- AGG> - TimestampedKeyValueStore<K,,- V> - TimestampedWindowStore<K,,- V> - WindowStore<K,- V> 
 If the store is implemented as a persistent store, it must use the store name as directory name and write
 all data into this store directory.
 The store directory must be created with the state directory.
 The state directory can be obtained via #stateDir() using the
 ProcessorContext provided via init(...).
 
Using nested store directories within the state directory isolates different state stores. If a state store would write into the state directory directly, it might conflict with others state stores and thus, data might get corrupted and/or Streams might fail with an error. Furthermore, Kafka Streams relies on using the store name as store directory name to perform internal cleanup tasks.
This interface does not specify any query capabilities, which, of course, would be query engine specific. Instead, it just specifies the minimum functionality required to reload a storage engine from its changelog as well as basic lifecycle management.
- 
Method SummaryModifier and TypeMethodDescriptionvoidclose()Close the storage engine.voidflush()Flush any cached datadefault PositionReturns the position the state store is at with respect to the input topic/partitionsvoidinit(ProcessorContext context, StateStore root) Deprecated.Since 2.7.0.default voidinit(StateStoreContext context, StateStore root) Initializes this state store.booleanisOpen()Is this store open for reading and writingname()The name of this store.booleanReturn if the storage is persistent or not.default <R> QueryResult<R>query(Query<R> query, PositionBound positionBound, QueryConfig config) Execute a query.
- 
Method Details- 
nameString name()The name of this store.- Returns:
- the storage name
 
- 
initDeprecated.Since 2.7.0. Callers should invokeinit(StateStoreContext, StateStore)instead. Implementers may choose to implement this method for backward compatibility or to throw an informative exception instead.Initializes this state store.The implementation of this function must register the root store in the context via the ProcessorContext.register(StateStore, StateRestoreCallback)function, where the firstStateStoreparameter should always be the passed-inrootobject, and the second parameter should be an object of user's implementation of theStateRestoreCallbackinterface used for restoring the state store from the changelog.Note that if the state store engine itself supports bulk writes, users can implement another interface BatchingStateRestoreCallbackwhich extendsStateRestoreCallbackto let users implement bulk-load restoration logic instead of restoring one record at a time.This method is not called if init(StateStoreContext, StateStore)is implemented.- Throws:
- IllegalStateException- If store gets registered after initialized is already finished
- StreamsException- if the store's change log does not contain the partition
 
- 
initInitializes this state store.The implementation of this function must register the root store in the context via the StateStoreContext.register(StateStore, StateRestoreCallback, CommitCallback)function, where the firstStateStoreparameter should always be the passed-inrootobject, and the second parameter should be an object of user's implementation of theStateRestoreCallbackinterface used for restoring the state store from the changelog.Note that if the state store engine itself supports bulk writes, users can implement another interface BatchingStateRestoreCallbackwhich extendsStateRestoreCallbackto let users implement bulk-load restoration logic instead of restoring one record at a time.- Throws:
- IllegalStateException- If store gets registered after initialized is already finished
- StreamsException- if the store's change log does not contain the partition
 
- 
flushvoid flush()Flush any cached data
- 
closevoid close()Close the storage engine. Note that this function needs to be idempotent since it may be called several times on the same state store.Users only need to implement this function but should NEVER need to call this api explicitly as it will be called by the library automatically when necessary 
- 
persistentboolean persistent()Return if the storage is persistent or not.- Returns:
- trueif the storage is persistent—- falseotherwise
 
- 
isOpenboolean isOpen()Is this store open for reading and writing- Returns:
- trueif the store is open
 
- 
query@Evolving default <R> QueryResult<R> query(Query<R> query, PositionBound positionBound, QueryConfig config) Execute a query. Returns a QueryResult containing either result data or a failure.If the store doesn't know how to handle the given query, the result shall be a FailureReason.UNKNOWN_QUERY_TYPE. If the store couldn't satisfy the given position bound, the result shall be aFailureReason.NOT_UP_TO_BOUND.Note to store implementers: if your store does not support position tracking, you can correctly respond FailureReason.NOT_UP_TO_BOUNDif the argument is anything butPositionBound.unbounded(). Be sure to explain in the failure message that bounded positions are not supported.- Type Parameters:
- R- The result type
- Parameters:
- query- The query to execute
- positionBound- The position the store must be at or past
- config- Per query configuration parameters, such as whether the store should collect detailed execution info for the query
 
- 
getPositionReturns the position the state store is at with respect to the input topic/partitions
 
-