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 VersionedBytesStoreSupplierpersistentVersionedKeyValueStore(String name, Duration historyRetention) Create a persistent versioned key-value storeVersionedBytesStoreSupplier.static VersionedBytesStoreSupplierpersistentVersionedKeyValueStore(String name, Duration historyRetention, Duration segmentInterval) Create a persistent versioned key-value storeVersionedBytesStoreSupplier.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<VersionedKeyValueStore<K, V>> versionedKeyValueStoreBuilder(VersionedBytesStoreSupplier supplier, Serde<K> keySerde, Serde<V> valueSerde) Creates aStoreBuilderthat can be used to build aVersionedKeyValueStore.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 aTimestampedKeyValueStoreorVersionedKeyValueStoreyou should usepersistentTimestampedKeyValueStore(String)orpersistentVersionedKeyValueStore(String, Duration), respectively, 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 aKeyValueStoreor aVersionedKeyValueStoreyou should usepersistentKeyValueStore(String)orpersistentVersionedKeyValueStore(String, Duration), respectively, 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
 
- 
persistentVersionedKeyValueStorepublic static VersionedBytesStoreSupplier persistentVersionedKeyValueStore(String name, Duration historyRetention) Create a persistent versioned key-value storeVersionedBytesStoreSupplier.This store supplier can be passed into a versionedKeyValueStoreBuilder(VersionedBytesStoreSupplier, Serde, Serde).Note that it is not safe to change the value of historyRetentionbetween application restarts without clearing local state from application instances, as this may cause incorrect values to be read from the state store if it impacts the underlying storage format.- Parameters:
- name- name of the store (cannot be- null)
- historyRetention- length of time that old record versions are available for query (cannot be negative). If a timestamp bound provided to- VersionedKeyValueStore.get(Object, long)is older than this specified history retention, then the get operation will not return data. This parameter also determines the "grace period" after which out-of-order writes will no longer be accepted.
- Returns:
- an instance of VersionedBytesStoreSupplier
- Throws:
- IllegalArgumentException- if- historyRetentioncan't be represented as- long milliseconds
 
- 
persistentVersionedKeyValueStorepublic static VersionedBytesStoreSupplier persistentVersionedKeyValueStore(String name, Duration historyRetention, Duration segmentInterval) Create a persistent versioned key-value storeVersionedBytesStoreSupplier.This store supplier can be passed into a versionedKeyValueStoreBuilder(VersionedBytesStoreSupplier, Serde, Serde).Note that it is not safe to change the value of segmentIntervalbetween application restarts without clearing local state from application instances, as this may cause incorrect values to be read from the state store otherwise.- Parameters:
- name- name of the store (cannot be- null)
- historyRetention- length of time that old record versions are available for query (cannot be negative). If a timestamp bound provided to- VersionedKeyValueStore.get(Object, long)is older than this specified history retention, then the get operation will not return data. This parameter also determines the "grace period" after which out-of-order writes will no longer be accepted.
- segmentInterval- size of segments for storing old record versions (must be positive). Old record versions for the same key in a single segment are stored (updated and accessed) together. The only impact of this parameter is performance. If segments are large and a workload results in many record versions for the same key being collected in a single segment, performance may degrade as a result. On the other hand, historical reads (which access older segments) and out-of-order writes may slow down if there are too many segments.
- Returns:
- an instance of VersionedBytesStoreSupplier
- Throws:
- IllegalArgumentException- if- historyRetentionor- segmentIntervalcan't be represented as- long milliseconds
 
- 
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.Note that it is not safe to change the value of retentionPeriodbetween application restarts without clearing local state from application instances, as this may cause incorrect values to be read from the state store if it impacts the underlying storage format.- 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.Note that it is not safe to change the value of retentionPeriodbetween application restarts without clearing local state from application instances, as this may cause incorrect values to be read from the state store if it impacts the underlying storage format.- 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.Note that it is not safe to change the value of retentionPeriodbetween application restarts without clearing local state from application instances, as this may cause incorrect values to be read from the state store if it impacts the underlying storage format.- 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
 
- 
versionedKeyValueStoreBuilderpublic static <K,V> StoreBuilder<VersionedKeyValueStore<K,V>> versionedKeyValueStoreBuilder(VersionedBytesStoreSupplier supplier, Serde<K> keySerde, Serde<V> valueSerde) Creates aStoreBuilderthat can be used to build aVersionedKeyValueStore.- Type Parameters:
- K- key type
- V- value type
- Parameters:
- supplier- a- VersionedBytesStoreSupplier(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 a deletion
- Returns:
- an instance of a StoreBuilderthat can build aVersionedKeyValueStore
 
- 
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
 
 
-