Package org.apache.kafka.streams.state
Interface ReadOnlyKeyValueStore<K,V>
-
- Type Parameters:
K
- the key typeV
- the value type
- All Known Subinterfaces:
KeyValueStore<K,V>
,TimestampedKeyValueStore<K,V>
public interface ReadOnlyKeyValueStore<K,V>
A key-value store that only supports read operations. Implementations should be thread-safe as concurrent reads and writes are expected.Please note that this contract defines the thread-safe read functionality only; it does not guarantee anything about whether the actual instance is writable by another thread, or whether it uses some locking mechanism under the hood. For this reason, making dependencies between the read and write operations on different StateStore instances can cause concurrency problems like deadlock.
-
-
Method Summary
All Methods Instance Methods Abstract Methods Default Methods Modifier and Type Method Description KeyValueIterator<K,V>
all()
Return an iterator over all keys in this store.long
approximateNumEntries()
Return an approximate count of key-value mappings in this store.V
get(K key)
Get the value corresponding to this key.default <PS extends Serializer<P>,P>
KeyValueIterator<K,V>prefixScan(P prefix, PS prefixKeySerializer)
Return an iterator over all keys with the specified prefix.KeyValueIterator<K,V>
range(K from, K to)
Get an iterator over a given range of keys.default KeyValueIterator<K,V>
reverseAll()
Return a reverse iterator over all keys in this store.default KeyValueIterator<K,V>
reverseRange(K from, K to)
Get a reverse iterator over a given range of keys.
-
-
-
Method Detail
-
get
V get(K key)
Get the value corresponding to this key.- Parameters:
key
- The key to fetch- Returns:
- The value or null if no value is found.
- Throws:
NullPointerException
- If null is used for key.InvalidStateStoreException
- if the store is not initialized
-
range
KeyValueIterator<K,V> range(K from, K to)
Get an iterator over a given range of keys. This iterator must be closed after use. The returned iterator must be safe fromConcurrentModificationException
s and must not return null values. Order is not guaranteed as bytes lexicographical ordering might not represent key order.- Parameters:
from
- The first key that could be in the range, where iteration starts from.to
- The last key that could be in the range, where iteration ends.- Returns:
- The iterator for this range, from smallest to largest bytes.
- Throws:
NullPointerException
- If null is used for from or to.InvalidStateStoreException
- if the store is not initialized
-
reverseRange
default KeyValueIterator<K,V> reverseRange(K from, K to)
Get a reverse iterator over a given range of keys. This iterator must be closed after use. The returned iterator must be safe fromConcurrentModificationException
s and must not return null values. Order is not guaranteed as bytes lexicographical ordering might not represent key order.- Parameters:
from
- The first key that could be in the range, where iteration ends.to
- The last key that could be in the range, where iteration starts from.- Returns:
- The reverse iterator for this range, from largest to smallest key bytes.
- Throws:
NullPointerException
- If null is used for from or to.InvalidStateStoreException
- if the store is not initialized
-
all
KeyValueIterator<K,V> all()
Return an iterator over all keys in this store. This iterator must be closed after use. The returned iterator must be safe fromConcurrentModificationException
s and must not return null values. Order is not guaranteed as bytes lexicographical ordering might not represent key order.- Returns:
- An iterator of all key/value pairs in the store, from smallest to largest bytes.
- Throws:
InvalidStateStoreException
- if the store is not initialized
-
reverseAll
default KeyValueIterator<K,V> reverseAll()
Return a reverse iterator over all keys in this store. This iterator must be closed after use. The returned iterator must be safe fromConcurrentModificationException
s and must not return null values. Order is not guaranteed as bytes lexicographical ordering might not represent key order.- Returns:
- An reverse iterator of all key/value pairs in the store, from largest to smallest key bytes.
- Throws:
InvalidStateStoreException
- if the store is not initialized
-
prefixScan
default <PS extends Serializer<P>,P> KeyValueIterator<K,V> prefixScan(P prefix, PS prefixKeySerializer)
Return an iterator over all keys with the specified prefix. Since the type of the prefix can be different from that of the key, a serializer to convert the prefix into the format in which the keys are stored in the stores needs to be passed to this method. The returned iterator must be safe fromConcurrentModificationException
s and must not return null values. SinceprefixScan()
relies on byte lexicographical ordering and not on the ordering of the key type, results for some types might be unexpected. For example, if the key type isInteger
, and the store contains keys [1, 2, 11, 13], then runningstore.prefixScan(1, new IntegerSerializer())
will return [1] and not [1,11,13]. In contrast, if the key type isString
the keys will be sorted [1, 11, 13, 2] in the store andstore.prefixScan(1, new StringSerializer())
will return [1,11,13]. In both casesprefixScan()
starts the scan at 1 and stops at 2.- Type Parameters:
PS
- Prefix Serializer typeP
- Prefix Type.- Parameters:
prefix
- The prefix.prefixKeySerializer
- Serializer for the Prefix key type- Returns:
- The iterator for keys having the specified prefix.
-
approximateNumEntries
long approximateNumEntries()
Return an approximate count of key-value mappings in this store.The count is not guaranteed to be exact in order to accommodate stores where an exact count is expensive to calculate.
- Returns:
- an approximate count of key-value mappings in the store.
- Throws:
InvalidStateStoreException
- if the store is not initialized
-
-