Package org.apache.kafka.streams.state
Class Stores
- java.lang.Object
-
- org.apache.kafka.streams.state.Stores
-
@Evolving public class Stores extends java.lang.ObjectFactory for creating state stores in Kafka Streams.When using the high-level DSL, i.e.,
StreamsBuilder, users createStoreSuppliers that can be further customized viaMaterialized. For example, a topic read asKTablecan be materialized into an in-memory store with custom key/value serdes and caching disabled:
When using the Processor API, i.e.,StreamsBuilder builder = new StreamsBuilder(); KeyValueBytesStoreSupplier storeSupplier = Stores.inMemoryKeyValueStore("queryable-store-name"); KTable<Long,String> table = builder.table( "topicName", Materialized.as(storeSupplier) .withKeySerde(Serdes.Long()) .withValueSerde(Serdes.String()) .withCachingDisabled());Topology, users createStoreBuilders that can be attached toProcessors. For example, you can create awindowedRocksDB store with custom changelog topic configuration like:Topology topology = new Topology(); topology.addProcessor("processorName", ...); Map<String,String> topicConfig = new HashMap<>(); StoreBuilder<WindowStore<Integer, Long>> storeBuilder = Stores .windowStoreBuilder( Stores.persistentWindowStore("queryable-store-name", ...), Serdes.Integer(), Serdes.Long()) .withLoggingEnabled(topicConfig); topology.addStateStore(storeBuilder, "processorName");
-
-
Nested Class Summary
Nested Classes Modifier and Type Class Description static interfaceStores.InMemoryKeyValueFactory<K,V>Deprecated.static interfaceStores.KeyValueFactory<K,V>static interfaceStores.PersistentKeyValueFactory<K,V>Deprecated.static classStores.StoreFactorystatic classStores.ValueFactory<K>The factory for creating off-heap key-value stores.
-
Constructor Summary
Constructors Constructor Description Stores()
-
Method Summary
All Methods Static Methods Concrete Methods Deprecated Methods Modifier and Type Method Description static Stores.StoreFactorycreate(java.lang.String name)static KeyValueBytesStoreSupplierinMemoryKeyValueStore(java.lang.String name)Create an in-memoryKeyValueBytesStoreSupplier.static <K,V>
StoreBuilder<KeyValueStore<K,V>>keyValueStoreBuilder(KeyValueBytesStoreSupplier supplier, org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valueSerde)Creates aStoreBuilderthan can be used to build aKeyValueStore.static KeyValueBytesStoreSupplierlruMap(java.lang.String name, int maxCacheSize)Create a LRU MapKeyValueBytesStoreSupplier.static KeyValueBytesStoreSupplierpersistentKeyValueStore(java.lang.String name)Create a persistentKeyValueBytesStoreSupplier.static SessionBytesStoreSupplierpersistentSessionStore(java.lang.String name, long retentionPeriod)Create a persistentSessionBytesStoreSupplier.static WindowBytesStoreSupplierpersistentWindowStore(java.lang.String name, long retentionPeriod, int numSegments, long windowSize, boolean retainDuplicates)Create a persistentWindowBytesStoreSupplier.static <K,V>
StoreBuilder<SessionStore<K,V>>sessionStoreBuilder(SessionBytesStoreSupplier supplier, org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valueSerde)Creates aStoreBuilderthat can be used to build aSessionStore.static <K,V>
StoreBuilder<WindowStore<K,V>>windowStoreBuilder(WindowBytesStoreSupplier supplier, org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valueSerde)Creates aStoreBuilderthat can be used to build aWindowStore.
-
-
-
Method Detail
-
persistentKeyValueStore
public static KeyValueBytesStoreSupplier persistentKeyValueStore(java.lang.String name)
Create a persistentKeyValueBytesStoreSupplier.- Parameters:
name- name of the store (cannot benull)- Returns:
- an instance of a
KeyValueBytesStoreSupplierthat can be used to build a persistent store
-
inMemoryKeyValueStore
public static KeyValueBytesStoreSupplier inMemoryKeyValueStore(java.lang.String name)
Create an in-memoryKeyValueBytesStoreSupplier.- Parameters:
name- name of the store (cannot benull)- Returns:
- an instance of a
KeyValueBytesStoreSupplierthan can be used to build an in-memory store
-
lruMap
public static KeyValueBytesStoreSupplier lruMap(java.lang.String name, int maxCacheSize)
Create a LRU MapKeyValueBytesStoreSupplier.- Parameters:
name- name of the store (cannot benull)maxCacheSize- maximum number of items in the LRU (cannot be negative)- Returns:
- an instance of a
KeyValueBytesStoreSupplierthat can be used to build an LRU Map based store
-
persistentWindowStore
public static WindowBytesStoreSupplier persistentWindowStore(java.lang.String name, long retentionPeriod, int numSegments, long windowSize, boolean retainDuplicates)
Create a persistentWindowBytesStoreSupplier.- Parameters:
name- name of the store (cannot benull)retentionPeriod- length of time to retain data in the store (cannot be negative)numSegments- number of db segments (cannot be zero or negative)windowSize- size of the windows (cannot be negative)retainDuplicates- whether or not to retain duplicates.- Returns:
- an instance of
WindowBytesStoreSupplier
-
persistentSessionStore
public static SessionBytesStoreSupplier persistentSessionStore(java.lang.String name, long retentionPeriod)
Create a persistentSessionBytesStoreSupplier.- Parameters:
name- name of the store (cannot benull)retentionPeriod- length ot time to retain data in the store (cannot be negative)- Returns:
- an instance of a
SessionBytesStoreSupplier
-
windowStoreBuilder
public static <K,V> StoreBuilder<WindowStore<K,V>> windowStoreBuilder(WindowBytesStoreSupplier supplier, org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valueSerde)
Creates aStoreBuilderthat can be used to build aWindowStore.- Type Parameters:
K- key typeV- value type- Parameters:
supplier- aWindowBytesStoreSupplier(cannot benull)keySerde- the key serde to usevalueSerde- the value serde to use; if the serialized bytes is null for put operations, it is treated as delete- Returns:
- an instance of
StoreBuilderthan can build aWindowStore
-
keyValueStoreBuilder
public static <K,V> StoreBuilder<KeyValueStore<K,V>> keyValueStoreBuilder(KeyValueBytesStoreSupplier supplier, org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valueSerde)
Creates aStoreBuilderthan can be used to build aKeyValueStore.- Type Parameters:
K- key typeV- value type- Parameters:
supplier- aKeyValueBytesStoreSupplier(cannot benull)keySerde- the key serde to usevalueSerde- the value serde to use; if the serialized bytes is null for put operations, it is treated as delete- Returns:
- an instance of a
StoreBuilderthat can build aKeyValueStore
-
sessionStoreBuilder
public static <K,V> StoreBuilder<SessionStore<K,V>> sessionStoreBuilder(SessionBytesStoreSupplier supplier, org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valueSerde)
Creates aStoreBuilderthat can be used to build aSessionStore.- Type Parameters:
K- key typeV- value type- Parameters:
supplier- aSessionBytesStoreSupplier(cannot benull)keySerde- the key serde to usevalueSerde- the value serde to use; if the serialized bytes is null for put operations, it is treated as delete- Returns:
- an instance of
StoreBuilderthan can build aSessionStore
-
create
@Deprecated public static Stores.StoreFactory create(java.lang.String name)
Deprecated.Begin to create a newStateStoreSupplierinstance.- Parameters:
name- the name of the store- Returns:
- the factory that can be used to specify other options or configurations for the store; never null
-
-