Interface WindowStore<K,V>
-
- Type Parameters:
K
- Type of keysV
- Type of values
- All Superinterfaces:
ReadOnlyWindowStore<K,V>
,StateStore
- All Known Subinterfaces:
TimestampedWindowStore<K,V>
public interface WindowStore<K,V> extends StateStore, ReadOnlyWindowStore<K,V>
Interface for storing the aggregated values of fixed-size time windows.Note, that the stores' physical key type is
Windowed<K>
.
-
-
Method Summary
All Methods Instance Methods Abstract Methods Default Methods Deprecated Methods Modifier and Type Method Description default 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).WindowStoreIterator<V>
fetch(K key, long timeFrom, long timeTo)
Get all the key-value pairs with the given key and the time range from all the existing windows.default WindowStoreIterator<V>
fetch(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.KeyValueIterator<Windowed<K>,V>
fetch(K keyFrom, K keyTo, long timeFrom, long timeTo)
Get all the key-value pairs in the given key range and time range from all the existing windows.default KeyValueIterator<Windowed<K>,V>
fetch(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.KeyValueIterator<Windowed<K>,V>
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>
fetchAll(Instant timeFrom, Instant timeTo)
Gets all the key-value pairs that belong to the windows within in the given time range.void
put(K key, V value)
Deprecated.as timestamp is not provided for the key-value pair, this causes inconsistency to identify the window frame to which the key belongs.void
put(K key, V value, long windowStartTimestamp)
Put a key-value pair into the window with given window start timestamp-
Methods inherited from interface org.apache.kafka.streams.state.ReadOnlyWindowStore
all, backwardAll, fetch
-
Methods inherited from interface org.apache.kafka.streams.processor.StateStore
close, flush, init, init, isOpen, name, persistent
-
-
-
-
Method Detail
-
put
@Deprecated void put(K key, V value)
Deprecated.as timestamp is not provided for the key-value pair, this causes inconsistency to identify the window frame to which the key belongs. Useput(Object, Object, long)
instead.Use the current record timestamp as thewindowStartTimestamp
and delegate toput(Object, Object, long)
.It's highly recommended to use
put(Object, Object, long)
instead, as the record timestamp is unlikely to be the correct windowStartTimestamp in general.- Parameters:
key
- The key to associate the value tovalue
- The value to update, it can be null; if the serialized bytes are also null it is interpreted as delete- Throws:
NullPointerException
- if the given key isnull
-
put
void put(K key, V value, long windowStartTimestamp)
Put 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 tovalue
- The value; can be nullwindowStartTimestamp
- The timestamp of the beginning of the window to put the key/value into- Throws:
NullPointerException
- if the given key isnull
-
fetch
WindowStoreIterator<V> fetch(K key, long timeFrom, long timeTo)
Get 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.
- Specified by:
fetch
in interfaceReadOnlyWindowStore<K,V>
- Parameters:
key
- the key to fetchtimeFrom
- 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 initializedNullPointerException
- if the given key isnull
-
fetch
default WindowStoreIterator<V> fetch(K key, Instant timeFrom, Instant timeTo) throws IllegalArgumentException
Description copied from interface:ReadOnlyWindowStore
Get 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:
fetch
in interfaceReadOnlyWindowStore<K,V>
- Parameters:
key
- the key to fetchtimeFrom
- 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 aslong milliseconds
-
backwardFetch
default WindowStoreIterator<V> backwardFetch(K key, long timeFrom, long timeTo)
-
backwardFetch
default WindowStoreIterator<V> backwardFetch(K key, Instant timeFrom, Instant timeTo) throws IllegalArgumentException
Description copied from interface:ReadOnlyWindowStore
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).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:
backwardFetch
in interfaceReadOnlyWindowStore<K,V>
- Parameters:
key
- the key to fetchtimeFrom
- 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 aslong milliseconds
-
fetch
KeyValueIterator<Windowed<K>,V> fetch(K keyFrom, K keyTo, long timeFrom, long timeTo)
Get 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:
fetch
in interfaceReadOnlyWindowStore<K,V>
- Parameters:
keyFrom
- the first key in the rangekeyTo
- the last key in the rangetimeFrom
- 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 initializedNullPointerException
- if one of the given keys isnull
-
fetch
default KeyValueIterator<Windowed<K>,V> fetch(K keyFrom, K keyTo, Instant timeFrom, Instant timeTo) throws IllegalArgumentException
Description copied from interface:ReadOnlyWindowStore
Get 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:
fetch
in interfaceReadOnlyWindowStore<K,V>
- Parameters:
keyFrom
- the first key in the rangekeyTo
- the last key in the rangetimeFrom
- 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 aslong milliseconds
-
backwardFetch
default KeyValueIterator<Windowed<K>,V> backwardFetch(K keyFrom, K keyTo, long timeFrom, long timeTo)
-
backwardFetch
default KeyValueIterator<Windowed<K>,V> backwardFetch(K keyFrom, K keyTo, Instant timeFrom, Instant timeTo) throws IllegalArgumentException
Description copied from interface:ReadOnlyWindowStore
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).This iterator must be closed after use.
- Specified by:
backwardFetch
in interfaceReadOnlyWindowStore<K,V>
- Parameters:
keyFrom
- the first key in the rangekeyTo
- the last key in the rangetimeFrom
- 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 aslong milliseconds
-
fetchAll
KeyValueIterator<Windowed<K>,V> fetchAll(long timeFrom, long timeTo)
Gets all the key-value pairs that belong to the windows within in the given time range.- Specified by:
fetchAll
in interfaceReadOnlyWindowStore<K,V>
- 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
-
fetchAll
default KeyValueIterator<Windowed<K>,V> fetchAll(Instant timeFrom, Instant timeTo) throws IllegalArgumentException
Description copied from interface:ReadOnlyWindowStore
Gets all the key-value pairs that belong to the windows within in the given time range.- Specified by:
fetchAll
in interfaceReadOnlyWindowStore<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 aslong milliseconds
-
backwardFetchAll
default KeyValueIterator<Windowed<K>,V> backwardFetchAll(long timeFrom, long timeTo)
-
backwardFetchAll
default KeyValueIterator<Windowed<K>,V> backwardFetchAll(Instant timeFrom, Instant timeTo) throws IllegalArgumentException
Description copied from interface:ReadOnlyWindowStore
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).- Specified by:
backwardFetchAll
in interfaceReadOnlyWindowStore<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:
- an 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 aslong milliseconds
-
-