Class Stores
 When using the high-level DSL, i.e., StreamsBuilder, users create
 StoreSuppliers that can be further customized via
 Materialized.
 For example, a topic read as KTable can be materialized into an
 in-memory store with custom key/value serdes and caching disabled:
 
 StreamsBuilder builder = new StreamsBuilder();
 KeyValueBytesStoreSupplier storeSupplier = Stores.inMemoryKeyValueStore("queryable-store-name");
 KTable<Long,String> table = builder.table(
   "topicName",
   Materialized.<Long,String>as(storeSupplier)
               .withKeySerde(Serdes.Long())
               .withValueSerde(Serdes.String())
               .withCachingDisabled());
 Topology, users create
 StoreBuilders that can be attached to Processors.
 For example, you can create a windowed RocksDB 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");
 - 
Constructor SummaryConstructors
- 
Method SummaryModifier and TypeMethodDescriptionstatic KeyValueBytesStoreSupplierinMemoryKeyValueStore(String name) Create an in-memoryKeyValueBytesStoreSupplier.static SessionBytesStoreSupplierinMemorySessionStore(String name, Duration retentionPeriod) Create an in-memorySessionBytesStoreSupplier.static WindowBytesStoreSupplierinMemoryWindowStore(String name, Duration retentionPeriod, Duration windowSize, boolean retainDuplicates) Create an in-memoryWindowBytesStoreSupplier.static <K,V> StoreBuilder<KeyValueStore<K, V>> keyValueStoreBuilder(KeyValueBytesStoreSupplier supplier, Serde<K> keySerde, Serde<V> valueSerde) Creates aStoreBuilderthat can be used to build aKeyValueStore.static KeyValueBytesStoreSupplierCreate a LRU MapKeyValueBytesStoreSupplier.static KeyValueBytesStoreSupplierCreate a persistentKeyValueBytesStoreSupplier.static SessionBytesStoreSupplierpersistentSessionStore(String name, Duration retentionPeriod) Create a persistentSessionBytesStoreSupplier.static KeyValueBytesStoreSupplierCreate a persistentKeyValueBytesStoreSupplier.static WindowBytesStoreSupplierpersistentTimestampedWindowStore(String name, Duration retentionPeriod, Duration windowSize, boolean retainDuplicates) Create a persistentWindowBytesStoreSupplier.static WindowBytesStoreSupplierpersistentWindowStore(String name, Duration retentionPeriod, Duration windowSize, boolean retainDuplicates) Create a persistentWindowBytesStoreSupplier.static <K,V> StoreBuilder<SessionStore<K, V>> sessionStoreBuilder(SessionBytesStoreSupplier supplier, Serde<K> keySerde, Serde<V> valueSerde) Creates aStoreBuilderthat can be used to build aSessionStore.static <K,V> StoreBuilder<TimestampedKeyValueStore<K, V>> timestampedKeyValueStoreBuilder(KeyValueBytesStoreSupplier supplier, Serde<K> keySerde, Serde<V> valueSerde) Creates aStoreBuilderthat can be used to build aTimestampedKeyValueStore.static <K,V> StoreBuilder<TimestampedWindowStore<K, V>> timestampedWindowStoreBuilder(WindowBytesStoreSupplier supplier, Serde<K> keySerde, Serde<V> valueSerde) Creates aStoreBuilderthat can be used to build aTimestampedWindowStore.static <K,V> StoreBuilder<WindowStore<K, V>> windowStoreBuilder(WindowBytesStoreSupplier supplier, Serde<K> keySerde, Serde<V> valueSerde) Creates aStoreBuilderthat can be used to build aWindowStore.
- 
Constructor Details- 
Storespublic Stores()
 
- 
- 
Method Details- 
persistentKeyValueStoreCreate a persistentKeyValueBytesStoreSupplier.This store supplier can be passed into a keyValueStoreBuilder(KeyValueBytesStoreSupplier, Serde, Serde). If you want to create aTimestampedKeyValueStoreyou should usepersistentTimestampedKeyValueStore(String)to create a store supplier instead.- Parameters:
- name- name of the store (cannot be- null)
- Returns:
- an instance of a KeyValueBytesStoreSupplierthat can be used to build a persistent key-value store
 
- 
persistentTimestampedKeyValueStoreCreate a persistentKeyValueBytesStoreSupplier.This store supplier can be passed into a timestampedKeyValueStoreBuilder(KeyValueBytesStoreSupplier, Serde, Serde). If you want to create aKeyValueStoreyou should usepersistentKeyValueStore(String)to create a store supplier instead.- Parameters:
- name- name of the store (cannot be- null)
- Returns:
- an instance of a KeyValueBytesStoreSupplierthat can be used to build a persistent key-(timestamp/value) store
 
- 
inMemoryKeyValueStoreCreate an in-memoryKeyValueBytesStoreSupplier.This store supplier can be passed into a keyValueStoreBuilder(KeyValueBytesStoreSupplier, Serde, Serde)ortimestampedKeyValueStoreBuilder(KeyValueBytesStoreSupplier, Serde, Serde).- Parameters:
- name- name of the store (cannot be- null)
- Returns:
- an instance of a KeyValueBytesStoreSupplierthan can be used to build an in-memory store
 
- 
lruMapCreate a LRU MapKeyValueBytesStoreSupplier.This store supplier can be passed into a keyValueStoreBuilder(KeyValueBytesStoreSupplier, Serde, Serde)ortimestampedKeyValueStoreBuilder(KeyValueBytesStoreSupplier, Serde, Serde).- Parameters:
- name- name of the store (cannot be- null)
- 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
- Throws:
- IllegalArgumentException- if- maxCacheSizeis negative
 
- 
persistentWindowStorepublic static WindowBytesStoreSupplier persistentWindowStore(String name, Duration retentionPeriod, Duration windowSize, boolean retainDuplicates) throws IllegalArgumentException Create a persistentWindowBytesStoreSupplier.This store supplier can be passed into a windowStoreBuilder(WindowBytesStoreSupplier, Serde, Serde). If you want to create aTimestampedWindowStoreyou should usepersistentTimestampedWindowStore(String, Duration, Duration, boolean)to create a store supplier instead.- Parameters:
- name- name of the store (cannot be- null)
- retentionPeriod- length of time to retain data in the store (cannot be negative) (note that the retention period must be at least long enough to contain the windowed data's entire life cycle, from window-start through window-end, and for the entire grace period)
- windowSize- size of the windows (cannot be negative)
- retainDuplicates- whether or not to retain duplicates. Turning this on will automatically disable caching and means that null values will be ignored.
- Returns:
- an instance of WindowBytesStoreSupplier
- Throws:
- IllegalArgumentException- if- retentionPeriodor- windowSizecan't be represented as- long milliseconds
- IllegalArgumentException- if- retentionPeriodis smaller than- windowSize
 
- 
persistentTimestampedWindowStorepublic static WindowBytesStoreSupplier persistentTimestampedWindowStore(String name, Duration retentionPeriod, Duration windowSize, boolean retainDuplicates) throws IllegalArgumentException Create a persistentWindowBytesStoreSupplier.This store supplier can be passed into a timestampedWindowStoreBuilder(WindowBytesStoreSupplier, Serde, Serde). If you want to create aWindowStoreyou should usepersistentWindowStore(String, Duration, Duration, boolean)to create a store supplier instead.- Parameters:
- name- name of the store (cannot be- null)
- retentionPeriod- length of time to retain data in the store (cannot be negative) (note that the retention period must be at least long enough to contain the windowed data's entire life cycle, from window-start through window-end, and for the entire grace period)
- windowSize- size of the windows (cannot be negative)
- retainDuplicates- whether or not to retain duplicates. Turning this on will automatically disable caching and means that null values will be ignored.
- Returns:
- an instance of WindowBytesStoreSupplier
- Throws:
- IllegalArgumentException- if- retentionPeriodor- windowSizecan't be represented as- long milliseconds
- IllegalArgumentException- if- retentionPeriodis smaller than- windowSize
 
- 
inMemoryWindowStorepublic static WindowBytesStoreSupplier inMemoryWindowStore(String name, Duration retentionPeriod, Duration windowSize, boolean retainDuplicates) throws IllegalArgumentException Create an in-memoryWindowBytesStoreSupplier.This store supplier can be passed into a windowStoreBuilder(WindowBytesStoreSupplier, Serde, Serde)ortimestampedWindowStoreBuilder(WindowBytesStoreSupplier, Serde, Serde).- Parameters:
- name- name of the store (cannot be- null)
- retentionPeriod- length of time to retain data in the store (cannot be negative) Note that the retention period must be at least long enough to contain the windowed data's entire life cycle, from window-start through window-end, and for the entire grace period.
- windowSize- size of the windows (cannot be negative)
- retainDuplicates- whether or not to retain duplicates. Turning this on will automatically disable caching and means that null values will be ignored.
- Returns:
- an instance of WindowBytesStoreSupplier
- Throws:
- IllegalArgumentException- if- retentionPeriodor- windowSizecan't be represented as- long milliseconds
- IllegalArgumentException- if- retentionPeriodis smaller than- windowSize
 
- 
persistentSessionStorepublic static SessionBytesStoreSupplier persistentSessionStore(String name, Duration retentionPeriod) Create a persistentSessionBytesStoreSupplier.- Parameters:
- name- name of the store (cannot be- null)
- retentionPeriod- length of time to retain data in the store (cannot be negative) (note that the retention period must be at least as long enough to contain the inactivity gap of the session and the entire grace period.)
- Returns:
- an instance of a SessionBytesStoreSupplier
 
- 
inMemorySessionStoreCreate an in-memorySessionBytesStoreSupplier.- Parameters:
- name- name of the store (cannot be- null)
- retentionPeriod- length ot time to retain data in the store (cannot be negative) (note that the retention period must be at least as long enough to contain the inactivity gap of the session and the entire grace period.)
- Returns:
- an instance of a SessionBytesStoreSupplier
 
- 
keyValueStoreBuilderpublic static <K,V> StoreBuilder<KeyValueStore<K,V>> keyValueStoreBuilder(KeyValueBytesStoreSupplier supplier, Serde<K> keySerde, Serde<V> valueSerde) Creates aStoreBuilderthat can be used to build aKeyValueStore.The provided supplier should not be a supplier for TimestampedKeyValueStores.- Type Parameters:
- K- key type
- V- value type
- Parameters:
- supplier- a- KeyValueBytesStoreSupplier(cannot be- null)
- keySerde- the key serde to use
- valueSerde- the value serde to use; if the serialized bytes is- nullfor put operations, it is treated as delete
- Returns:
- an instance of a StoreBuilderthat can build aKeyValueStore
 
- 
timestampedKeyValueStoreBuilderpublic static <K,V> StoreBuilder<TimestampedKeyValueStore<K,V>> timestampedKeyValueStoreBuilder(KeyValueBytesStoreSupplier supplier, Serde<K> keySerde, Serde<V> valueSerde) Creates aStoreBuilderthat can be used to build aTimestampedKeyValueStore.The provided supplier should not be a supplier for KeyValueStores. For this case, passed in timestamps will be dropped and not stored in the key-value-store. On read, no valid timestamp but a dummy timestamp will be returned.- Type Parameters:
- K- key type
- V- value type
- Parameters:
- supplier- a- KeyValueBytesStoreSupplier(cannot be- null)
- keySerde- the key serde to use
- valueSerde- the value serde to use; if the serialized bytes is- nullfor put operations, it is treated as delete
- Returns:
- an instance of a StoreBuilderthat can build aKeyValueStore
 
- 
windowStoreBuilderpublic static <K,V> StoreBuilder<WindowStore<K,V>> windowStoreBuilder(WindowBytesStoreSupplier supplier, Serde<K> keySerde, Serde<V> valueSerde) Creates aStoreBuilderthat can be used to build aWindowStore.The provided supplier should not be a supplier for TimestampedWindowStores.- Type Parameters:
- K- key type
- V- value type
- Parameters:
- supplier- a- WindowBytesStoreSupplier(cannot be- null)
- keySerde- the key serde to use
- valueSerde- the value serde to use; if the serialized bytes is- nullfor put operations, it is treated as delete
- Returns:
- an instance of StoreBuilderthan can build aWindowStore
 
- 
timestampedWindowStoreBuilderpublic static <K,V> StoreBuilder<TimestampedWindowStore<K,V>> timestampedWindowStoreBuilder(WindowBytesStoreSupplier supplier, Serde<K> keySerde, Serde<V> valueSerde) Creates aStoreBuilderthat can be used to build aTimestampedWindowStore.The provided supplier should not be a supplier for WindowStores. For this case, passed in timestamps will be dropped and not stored in the window-store. On read, no valid timestamp but a dummy timestamp will be returned.- Type Parameters:
- K- key type
- V- value type
- Parameters:
- supplier- a- WindowBytesStoreSupplier(cannot be- null)
- keySerde- the key serde to use
- valueSerde- the value serde to use; if the serialized bytes is- nullfor put operations, it is treated as delete
- Returns:
- an instance of StoreBuilderthat can build aTimestampedWindowStore
 
- 
sessionStoreBuilderpublic static <K,V> StoreBuilder<SessionStore<K,V>> sessionStoreBuilder(SessionBytesStoreSupplier supplier, Serde<K> keySerde, Serde<V> valueSerde) Creates aStoreBuilderthat can be used to build aSessionStore.- Type Parameters:
- K- key type
- V- value type
- Parameters:
- supplier- a- SessionBytesStoreSupplier(cannot be- null)
- keySerde- the key serde to use
- valueSerde- the value serde to use; if the serialized bytes is- nullfor put operations, it is treated as delete
- Returns:
- an instance of StoreBuilderthan can build aSessionStore
 
 
-