Interface StateRestoreListener
-
public interface StateRestoreListener
Class for listening to various states of the restoration process of a StateStore. When callingKafkaStreams.setGlobalStateRestoreListener(StateRestoreListener)
the passed instance is expected to be stateless since theStateRestoreListener
is shared across allStreamThread
instances. Users desiring stateful operations will need to provide synchronization internally in theStateRestorerListener
implementation. Note that this listener is only registered at the per-client level and users can base on thestoreName
parameter to define specific monitoring for differentStateStore
s. There is anotherStateRestoreCallback
interface which is registered via theProcessorContext.register(StateStore, StateRestoreCallback)
function per-store, and it is used to apply the fetched changelog records into the local state store during restoration. These two interfaces serve different restoration purposes and users should not try to implement both of them in a single class during state store registration. Incremental updates are exposed so users can estimate how much progress has been made.
-
-
Method Summary
All Methods Instance Methods Abstract Methods Modifier and Type Method Description void
onBatchRestored(TopicPartition topicPartition, String storeName, long batchEndOffset, long numRestored)
Method called after restoring a batch of records.void
onRestoreEnd(TopicPartition topicPartition, String storeName, long totalRestored)
Method called when restoring theStateStore
is complete.void
onRestoreStart(TopicPartition topicPartition, String storeName, long startingOffset, long endingOffset)
Method called at the very beginning ofStateStore
restoration.
-
-
-
Method Detail
-
onRestoreStart
void onRestoreStart(TopicPartition topicPartition, String storeName, long startingOffset, long endingOffset)
Method called at the very beginning ofStateStore
restoration.- Parameters:
topicPartition
- the TopicPartition containing the values to restorestoreName
- the name of the store undergoing restorationstartingOffset
- the starting offset of the entire restoration process for this TopicPartitionendingOffset
- the exclusive ending offset of the entire restoration process for this TopicPartition
-
onBatchRestored
void onBatchRestored(TopicPartition topicPartition, String storeName, long batchEndOffset, long numRestored)
Method called after restoring a batch of records. In this case the maximum size of the batch is whatever the value of the MAX_POLL_RECORDS is set to. This method is called after restoring each batch and it is advised to keep processing to a minimum. Any heavy processing will hold up recovering the next batch, hence slowing down the restore process as a whole. If you need to do any extended processing or connecting to an external service consider doing so asynchronously.- Parameters:
topicPartition
- the TopicPartition containing the values to restorestoreName
- the name of the store undergoing restorationbatchEndOffset
- the inclusive ending offset for the current restored batch for this TopicPartitionnumRestored
- the total number of records restored in this batch for this TopicPartition
-
onRestoreEnd
void onRestoreEnd(TopicPartition topicPartition, String storeName, long totalRestored)
Method called when restoring theStateStore
is complete.- Parameters:
topicPartition
- the TopicPartition containing the values to restorestoreName
- the name of the store just restoredtotalRestored
- the total number of records restored for this TopicPartition
-
-