Interface WindowStore<K,V> 
- Type Parameters:
- K- Type of keys
- V- Type of values
- All Superinterfaces:
- ReadOnlyWindowStore<K,,- V> - StateStore
- All Known Subinterfaces:
- TimestampedWindowStore<K,- V> 
 Note, that the stores' physical key type is Windowed<K>.
- 
Method SummaryModifier and TypeMethodDescriptiondefault WindowStoreIterator<V>backwardFetch(K key, long timeFrom, long timeTo) default WindowStoreIterator<V>backwardFetch(K key, Instant timeFrom, Instant timeTo) Get all the key-value pairs with the given key and the time range from all the existing windows in backward order with respect to time (from end to beginning of time).default KeyValueIterator<Windowed<K>,V> backwardFetch(K keyFrom, K keyTo, long timeFrom, long timeTo) default KeyValueIterator<Windowed<K>,V> backwardFetch(K keyFrom, K keyTo, Instant timeFrom, Instant timeTo) Get all the key-value pairs in the given key range and time range from all the existing windows in backward order with respect to time (from end to beginning of time).default KeyValueIterator<Windowed<K>,V> backwardFetchAll(long timeFrom, long timeTo) default KeyValueIterator<Windowed<K>,V> backwardFetchAll(Instant timeFrom, Instant timeTo) Gets all the key-value pairs that belong to the windows within in the given time range in backward order with respect to time (from end to beginning of time).Get all the key-value pairs with the given key and the time range from all the existing windows.default WindowStoreIterator<V>Get all the key-value pairs with the given key and the time range from all the existing windows.Get all the key-value pairs in the given key range and time range from all the existing windows.default KeyValueIterator<Windowed<K>,V> Get all the key-value pairs in the given key range and time range from all the existing windows.fetchAll(long timeFrom, long timeTo) Gets all the key-value pairs that belong to the windows within in the given time range.default KeyValueIterator<Windowed<K>,V> Gets all the key-value pairs that belong to the windows within in the given time range.voidPut a key-value pair into the window with given window start timestampMethods inherited from interface org.apache.kafka.streams.state.ReadOnlyWindowStoreall, backwardAll, fetchMethods inherited from interface org.apache.kafka.streams.processor.StateStoreclose, flush, getPosition, init, init, isOpen, name, persistent, query
- 
Method Details- 
putPut a key-value pair into the window with given window start timestampIf serialized value bytes are null it is interpreted as delete. Note that deletes will be ignored in the case of an underlying store that retains duplicates. - Parameters:
- key- The key to associate the value to
- value- The value; can be null
- windowStartTimestamp- The timestamp of the beginning of the window to put the key/value into
- Throws:
- InvalidStateStoreException- if the store is not initialized
- NullPointerException- if the given key is- null
 
- 
fetchGet all the key-value pairs with the given key and the time range from all the existing windows.This iterator must be closed after use. The time range is inclusive and applies to the starting timestamp of the window. For example, if we have the following windows: +-------------------------------+ | key | start time | end time | +-------+------------+----------+ | A | 10 | 20 | +-------+------------+----------+ | A | 15 | 25 | +-------+------------+----------+ | A | 20 | 30 | +-------+------------+----------+ | A | 25 | 35 | +-------------------------------- And we callstore.fetch("A", 10, 20)then the results will contain the first three windows from the table above, i.e., all those where 10 <= start time <= 20.For each key, the iterator guarantees ordering of windows, starting from the oldest/earliest available window to the newest/latest window. - Parameters:
- key- the key to fetch
- timeFrom- time range start (inclusive)
- timeTo- time range end (inclusive)
- Returns:
- an iterator over key-value pairs <timestamp, value>
- Throws:
- InvalidStateStoreException- if the store is not initialized
- NullPointerException- if the given key is- null
 
- 
fetchdefault WindowStoreIterator<V> fetch(K key, Instant timeFrom, Instant timeTo) throws IllegalArgumentException Description copied from interface:ReadOnlyWindowStoreGet all the key-value pairs with the given key and the time range from all the existing windows.This iterator must be closed after use. The time range is inclusive and applies to the starting timestamp of the window. For example, if we have the following windows: +-------------------------------+ | key | start time | end time | +-------+------------+----------+ | A | 10 | 20 | +-------+------------+----------+ | A | 15 | 25 | +-------+------------+----------+ | A | 20 | 30 | +-------+------------+----------+ | A | 25 | 35 | +-------------------------------- And we callstore.fetch("A", Instant.ofEpochMilli(10), Instant.ofEpochMilli(20))then the results will contain the first three windows from the table above, i.e., all those where 10 <= start time <= 20.For each key, the iterator guarantees ordering of windows, starting from the oldest/earliest available window to the newest/latest window. - Specified by:
- fetchin interface- ReadOnlyWindowStore<K,- V> 
- Parameters:
- key- the key to fetch
- timeFrom- time range start (inclusive), where iteration starts.
- timeTo- time range end (inclusive), where iteration ends.
- Returns:
- an iterator over key-value pairs <timestamp, value>, from beginning to end of time.
- Throws:
- IllegalArgumentException- if duration is negative or can't be represented as- long milliseconds
 
- 
backwardFetch
- 
backwardFetchdefault WindowStoreIterator<V> backwardFetch(K key, Instant timeFrom, Instant timeTo) throws IllegalArgumentException Description copied from interface:ReadOnlyWindowStoreGet all the key-value pairs with the given key and the time range from all the existing windows in backward order with respect to time (from end to beginning of time).This iterator must be closed after use. The time range is inclusive and applies to the starting timestamp of the window. For example, if we have the following windows: +-------------------------------+ | key | start time | end time | +-------+------------+----------+ | A | 10 | 20 | +-------+------------+----------+ | A | 15 | 25 | +-------+------------+----------+ | A | 20 | 30 | +-------+------------+----------+ | A | 25 | 35 | +-------------------------------- And we callstore.backwardFetch("A", Instant.ofEpochMilli(10), Instant.ofEpochMilli(20))then the results will contain the first three windows from the table above in backward order, i.e., all those where 10 <= start time <= 20.For each key, the iterator guarantees ordering of windows, starting from the newest/latest available window to the oldest/earliest window. - Specified by:
- backwardFetchin interface- ReadOnlyWindowStore<K,- V> 
- Parameters:
- key- the key to fetch
- timeFrom- time range start (inclusive), where iteration ends.
- timeTo- time range end (inclusive), where iteration starts.
- Returns:
- an iterator over key-value pairs <timestamp, value>, from end to beginning of time.
- Throws:
- IllegalArgumentException- if duration is negative or can't be represented as- long milliseconds
 
- 
fetchGet all the key-value pairs in the given key range and time range from all the existing windows.This iterator must be closed after use. - Parameters:
- keyFrom- the first key in the range A null value indicates a starting position from the first element in the store.
- keyTo- the last key in the range A null value indicates that the range ends with the last element in the store.
- timeFrom- time range start (inclusive)
- timeTo- time range end (inclusive)
- Returns:
- an iterator over windowed key-value pairs <Windowed<K>, value>
- Throws:
- InvalidStateStoreException- if the store is not initialized
 
- 
fetchdefault KeyValueIterator<Windowed<K>,V> fetch(K keyFrom, K keyTo, Instant timeFrom, Instant timeTo) throws IllegalArgumentException Description copied from interface:ReadOnlyWindowStoreGet all the key-value pairs in the given key range and time range from all the existing windows.This iterator must be closed after use. - Specified by:
- fetchin interface- ReadOnlyWindowStore<K,- V> 
- Parameters:
- keyFrom- the first key in the range A null value indicates a starting position from the first element in the store.
- keyTo- the last key in the range A null value indicates that the range ends with the last element in the store.
- timeFrom- time range start (inclusive), where iteration starts.
- timeTo- time range end (inclusive), where iteration ends.
- Returns:
- an iterator over windowed key-value pairs <Windowed<K>, value>, from beginning to end of time.
- Throws:
- IllegalArgumentException- if duration is negative or can't be represented as- long milliseconds
 
- 
backwardFetch
- 
backwardFetchdefault KeyValueIterator<Windowed<K>,V> backwardFetch(K keyFrom, K keyTo, Instant timeFrom, Instant timeTo) throws IllegalArgumentException Description copied from interface:ReadOnlyWindowStoreGet all the key-value pairs in the given key range and time range from all the existing windows in backward order with respect to time (from end to beginning of time).This iterator must be closed after use. - Specified by:
- backwardFetchin interface- ReadOnlyWindowStore<K,- V> 
- Parameters:
- keyFrom- the first key in the range A null value indicates a starting position from the first element in the store.
- keyTo- the last key in the range A null value indicates that the range ends with the last element in the store.
- timeFrom- time range start (inclusive), where iteration ends.
- timeTo- time range end (inclusive), where iteration starts.
- Returns:
- an iterator over windowed key-value pairs <Windowed<K>, value>, from end to beginning of time.
- Throws:
- IllegalArgumentException- if duration is negative or can't be represented as- long milliseconds
 
- 
fetchAllGets all the key-value pairs that belong to the windows within in the given time range.- Parameters:
- timeFrom- the beginning of the time slot from which to search (inclusive)
- timeTo- the end of the time slot from which to search (inclusive)
- Returns:
- an iterator over windowed key-value pairs <Windowed<K>, value>
- Throws:
- InvalidStateStoreException- if the store is not initialized
 
- 
fetchAlldefault KeyValueIterator<Windowed<K>,V> fetchAll(Instant timeFrom, Instant timeTo) throws IllegalArgumentException Description copied from interface:ReadOnlyWindowStoreGets all the key-value pairs that belong to the windows within in the given time range.- Specified by:
- fetchAllin interface- ReadOnlyWindowStore<K,- V> 
- Parameters:
- timeFrom- the beginning of the time slot from which to search (inclusive), where iteration starts.
- timeTo- the end of the time slot from which to search (inclusive), where iteration ends.
- Returns:
- an iterator over windowed key-value pairs <Windowed<K>, value>, from beginning to end of time.
- Throws:
- IllegalArgumentException- if duration is negative or can't be represented as- long milliseconds
 
- 
backwardFetchAll
- 
backwardFetchAlldefault KeyValueIterator<Windowed<K>,V> backwardFetchAll(Instant timeFrom, Instant timeTo) throws IllegalArgumentException Description copied from interface:ReadOnlyWindowStoreGets all the key-value pairs that belong to the windows within in the given time range in backward order with respect to time (from end to beginning of time).- Specified by:
- backwardFetchAllin interface- ReadOnlyWindowStore<K,- V> 
- Parameters:
- timeFrom- the beginning of the time slot from which to search (inclusive), where iteration ends.
- timeTo- the end of the time slot from which to search (inclusive), where iteration starts.
- Returns:
- a backward iterator over windowed key-value pairs <Windowed<K>, value>, from end to beginning of time.
- Throws:
- IllegalArgumentException- if duration is negative or can't be represented as- long milliseconds
 
 
-