Package org.apache.kafka.streams.kstream
Class Materialized<K,V,S extends StateStore>  
java.lang.Object
org.apache.kafka.streams.kstream.Materialized<K,V,S>  
- Type Parameters:
- K- type of record key
- V- type of record value
- S- type of state store (note: state stores always have key/value types- <Bytes,byte[]>
Used to describe how a 
StateStore should be materialized.
 You can either provide a custom StateStore backend through one of the provided methods accepting a supplier
 or use the default RocksDB backends by providing just a store name.
 
 For example, you can read a topic as KTable and force a state store materialization to access the content
 via Interactive Queries API:
 
 StreamsBuilder builder = new StreamsBuilder();
 KTable<Integer, Integer> table = builder.table(
   "topicName",
   Materialized.as("queryable-store-name"));
 - See Also:
- 
Nested Class SummaryNested Classes
- 
Field SummaryFieldsModifier and TypeFieldDescriptionprotected booleanprotected booleanprotected Durationprotected Stringprotected StoreSupplier<S>
- 
Constructor SummaryConstructorsModifierConstructorDescriptionprotectedMaterialized(Materialized<K, V, S> materialized) Copy constructor.
- 
Method SummaryModifier and TypeMethodDescriptionstatic <K,V, S extends StateStore> 
 Materialized<K,V, S> Materialize aStateStorewith the given name.static <K,V, S extends StateStore> 
 Materialized<K,V, S> as(Materialized.StoreType storeType) Materialize aStateStorewith the givenMaterialized.StoreType.static <K,V> Materialized<K, V, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> as(KeyValueBytesStoreSupplier supplier) Materialize aKeyValueStoreusing the providedKeyValueBytesStoreSupplier.static <K,V> Materialized<K, V, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> as(SessionBytesStoreSupplier supplier) Materialize aSessionStoreusing the providedSessionBytesStoreSupplier.static <K,V> Materialized<K, V, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> as(WindowBytesStoreSupplier supplier) Materialize aWindowStoreusing the providedWindowBytesStoreSupplier.static <K,V, S extends StateStore> 
 Materialized<K,V, S> Materialize aStateStorewith the provided key and valueSerdes.Materialized<K,V, S> Disable caching for the materializedStateStore.Materialized<K,V, S> Enable caching for the materializedStateStore.Materialized<K,V, S> withKeySerde(Serde<K> keySerde) Set the keySerde the materializeStateStorewill use.Materialized<K,V, S> Disable change logging for the materializedStateStore.Materialized<K,V, S> withLoggingEnabled(Map<String, String> config) Indicates that a changelog should be created for the store.Materialized<K,V, S> withRetention(Duration retention) Configure retention period for window and session stores.Materialized<K,V, S> withStoreType(Materialized.StoreType storeType) Set the type of the materializedStateStore.Materialized<K,V, S> withValueSerde(Serde<V> valueSerde) Set the valueSerde the materializedStateStorewill use.
- 
Field Details- 
storeSupplier
- 
storeName
- 
valueSerde
- 
keySerde
- 
loggingEnabledprotected boolean loggingEnabled
- 
cachingEnabledprotected boolean cachingEnabled
- 
topicConfig
- 
retention
- 
storeType
 
- 
- 
Constructor Details- 
MaterializedCopy constructor.- Parameters:
- materialized- the- Materializedinstance to copy.
 
 
- 
- 
Method Details- 
asMaterialize aStateStorewith the givenMaterialized.StoreType.- Type Parameters:
- K- key type of the store
- V- value type of the store
- S- type of the- StateStore
- Parameters:
- storeType- the type of the state store
- Returns:
- a new Materializedinstance with the given storeName
 
- 
asMaterialize aStateStorewith the given name.- Type Parameters:
- K- key type of the store
- V- value type of the store
- S- type of the- StateStore
- Parameters:
- storeName- the name of the underlying- KTablestate store; valid characters are ASCII alphanumerics, '.', '_' and '-'.
- Returns:
- a new Materializedinstance with the given storeName
 
- 
aspublic static <K,V> Materialized<K,V, asWindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> (WindowBytesStoreSupplier supplier) Materialize aWindowStoreusing the providedWindowBytesStoreSupplier. Important: Custom subclasses are allowed here, but they should respect the retention contract: Window stores are required to retain windows at least as long as (window size + window grace period). Stores constructed viaStoresalready satisfy this contract.- Type Parameters:
- K- key type of the store
- V- value type of the store
- Parameters:
- supplier- the- WindowBytesStoreSupplierused to materialize the store
- Returns:
- a new Materializedinstance with the given supplier
 
- 
aspublic static <K,V> Materialized<K,V, asSessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> (SessionBytesStoreSupplier supplier) Materialize aSessionStoreusing the providedSessionBytesStoreSupplier. Important: Custom subclasses are allowed here, but they should respect the retention contract: Session stores are required to retain windows at least as long as (session inactivity gap + session grace period). Stores constructed viaStoresalready satisfy this contract.- Type Parameters:
- K- key type of the store
- V- value type of the store
- Parameters:
- supplier- the- SessionBytesStoreSupplierused to materialize the store
- Returns:
- a new Materializedinstance with the given sup plier
 
- 
aspublic static <K,V> Materialized<K,V, asKeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> (KeyValueBytesStoreSupplier supplier) Materialize aKeyValueStoreusing the providedKeyValueBytesStoreSupplier.- Type Parameters:
- K- key type of the store
- V- value type of the store
- Parameters:
- supplier- the- KeyValueBytesStoreSupplierused to materialize the store
- Returns:
- a new Materializedinstance with the given supplier
 
- 
withpublic static <K,V, Materialized<K,S extends StateStore> V, withS> (Serde<K> keySerde, Serde<V> valueSerde) Materialize aStateStorewith the provided key and valueSerdes. An internal name will be used for the store.- Type Parameters:
- K- key type
- V- value type
- S- store type
- Parameters:
- keySerde- the key- Serdeto use. If the- Serdeis null, then the default key serde from configs will be used
- valueSerde- the value- Serdeto use. If the- Serdeis null, then the default value serde from configs will be used
- Returns:
- a new Materializedinstance with the given key and value serdes
 
- 
withValueSerdeSet the valueSerde the materializedStateStorewill use.
- 
withKeySerdeSet the keySerde the materializeStateStorewill use.
- 
withLoggingEnabledIndicates that a changelog should be created for the store. The changelog will be created with the provided configs.Note: Any unrecognized configs will be ignored. - Parameters:
- config- any configs that should be applied to the changelog
- Returns:
- itself
 
- 
withLoggingDisabledDisable change logging for the materializedStateStore.- Returns:
- itself
 
- 
withCachingEnabledEnable caching for the materializedStateStore.- Returns:
- itself
 
- 
withCachingDisabledDisable caching for the materializedStateStore.- Returns:
- itself
 
- 
withRetentionConfigure retention period for window and session stores. Ignored for key/value stores. Overridden by pre-configured store suppliers (as(SessionBytesStoreSupplier)oras(WindowBytesStoreSupplier)). 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. If not specified, the retention period would be set as the window length (from window-start through window-end) plus the grace period.- Parameters:
- retention- the retention time
- Returns:
- itself
- Throws:
- IllegalArgumentException- if retention is negative or can't be represented as- long milliseconds
 
- 
withStoreTypepublic Materialized<K,V, withStoreTypeS> (Materialized.StoreType storeType) throws IllegalArgumentException Set the type of the materializedStateStore.- Parameters:
- storeType- the store type- Materialized.StoreTypeto use.
- Returns:
- itself
- Throws:
- IllegalArgumentException- if store supplier is also pre-configured
 
 
-