@InterfaceStability.Evolving public class Stores extends java.lang.Object
When using the high-level DSL, i.e., StreamsBuilder
, users create
StoreSupplier
s 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());
When using the Processor API, i.e., Topology
, users create
StoreBuilder
s that can be attached to Processor
s.
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 and Description |
---|
Stores() |
Modifier and Type | Method and Description |
---|---|
static KeyValueBytesStoreSupplier |
inMemoryKeyValueStore(java.lang.String name)
Create an in-memory
KeyValueBytesStoreSupplier . |
static <K,V> StoreBuilder<KeyValueStore<K,V>> |
keyValueStoreBuilder(KeyValueBytesStoreSupplier supplier,
Serde<K> keySerde,
Serde<V> valueSerde)
Creates a
StoreBuilder than can be used to build a KeyValueStore . |
static KeyValueBytesStoreSupplier |
lruMap(java.lang.String name,
int maxCacheSize)
Create a LRU Map
KeyValueBytesStoreSupplier . |
static KeyValueBytesStoreSupplier |
persistentKeyValueStore(java.lang.String name)
Create a persistent
KeyValueBytesStoreSupplier . |
static SessionBytesStoreSupplier |
persistentSessionStore(java.lang.String name,
java.time.Duration retentionPeriod)
Create a persistent
SessionBytesStoreSupplier . |
static SessionBytesStoreSupplier |
persistentSessionStore(java.lang.String name,
long retentionPeriod)
Deprecated.
since 2.1 Use
persistentSessionStore(String, Duration) instead |
static WindowBytesStoreSupplier |
persistentWindowStore(java.lang.String name,
java.time.Duration retentionPeriod,
java.time.Duration windowSize,
boolean retainDuplicates)
Create a persistent
WindowBytesStoreSupplier . |
static WindowBytesStoreSupplier |
persistentWindowStore(java.lang.String name,
long retentionPeriod,
int numSegments,
long windowSize,
boolean retainDuplicates)
Deprecated.
since 2.1 Use
persistentWindowStore(String, Duration, Duration, boolean) instead |
static <K,V> StoreBuilder<SessionStore<K,V>> |
sessionStoreBuilder(SessionBytesStoreSupplier supplier,
Serde<K> keySerde,
Serde<V> valueSerde)
Creates a
StoreBuilder that can be used to build a SessionStore . |
static <K,V> StoreBuilder<WindowStore<K,V>> |
windowStoreBuilder(WindowBytesStoreSupplier supplier,
Serde<K> keySerde,
Serde<V> valueSerde)
Creates a
StoreBuilder that can be used to build a WindowStore . |
public static KeyValueBytesStoreSupplier persistentKeyValueStore(java.lang.String name)
KeyValueBytesStoreSupplier
.name
- name of the store (cannot be null
)KeyValueBytesStoreSupplier
that can be used
to build a persistent storepublic static KeyValueBytesStoreSupplier inMemoryKeyValueStore(java.lang.String name)
KeyValueBytesStoreSupplier
.name
- name of the store (cannot be null
)KeyValueBytesStoreSupplier
than can be used to
build an in-memory storepublic static KeyValueBytesStoreSupplier lruMap(java.lang.String name, int maxCacheSize)
KeyValueBytesStoreSupplier
.name
- name of the store (cannot be null
)maxCacheSize
- maximum number of items in the LRU (cannot be negative)KeyValueBytesStoreSupplier
that can be used to build
an LRU Map based store@Deprecated public static WindowBytesStoreSupplier persistentWindowStore(java.lang.String name, long retentionPeriod, int numSegments, long windowSize, boolean retainDuplicates)
persistentWindowStore(String, Duration, Duration, boolean)
insteadWindowBytesStoreSupplier
.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.numSegments
- number of db segments (cannot be zero or negative)windowSize
- size of the windows that are stored (cannot be negative). Note: the window size
is not stored with the records, so this value is used to compute the keys that
the store returns. No effort is made to validate this parameter, so you must be
careful to set it the same as the windowed keys you're actually storing.retainDuplicates
- whether or not to retain duplicates.WindowBytesStoreSupplier
public static WindowBytesStoreSupplier persistentWindowStore(java.lang.String name, java.time.Duration retentionPeriod, java.time.Duration windowSize, boolean retainDuplicates) throws java.lang.IllegalArgumentException
WindowBytesStoreSupplier
.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.WindowBytesStoreSupplier
java.lang.IllegalArgumentException
- if retentionPeriod
or windowSize
can't be represented as long milliseconds
@Deprecated public static SessionBytesStoreSupplier persistentSessionStore(java.lang.String name, long retentionPeriod)
persistentSessionStore(String, Duration)
insteadSessionBytesStoreSupplier
.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 long enough to contain the
windowed data's entire life cycle, from window-start through window-end,
and for the entire grace period.SessionBytesStoreSupplier
public static SessionBytesStoreSupplier persistentSessionStore(java.lang.String name, java.time.Duration retentionPeriod)
SessionBytesStoreSupplier
.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 long enough to contain the
windowed data's entire life cycle, from window-start through window-end,
and for the entire grace period.SessionBytesStoreSupplier
public static <K,V> StoreBuilder<WindowStore<K,V>> windowStoreBuilder(WindowBytesStoreSupplier supplier, Serde<K> keySerde, Serde<V> valueSerde)
StoreBuilder
that can be used to build a WindowStore
.K
- key typeV
- value typesupplier
- a WindowBytesStoreSupplier
(cannot be null
)keySerde
- the key serde to usevalueSerde
- the value serde to use; if the serialized bytes is null for put operations,
it is treated as deleteStoreBuilder
than can build a WindowStore
public static <K,V> StoreBuilder<KeyValueStore<K,V>> keyValueStoreBuilder(KeyValueBytesStoreSupplier supplier, Serde<K> keySerde, Serde<V> valueSerde)
StoreBuilder
than can be used to build a KeyValueStore
.K
- key typeV
- value typesupplier
- a KeyValueBytesStoreSupplier
(cannot be null
)keySerde
- the key serde to usevalueSerde
- the value serde to use; if the serialized bytes is null for put operations,
it is treated as deleteStoreBuilder
that can build a KeyValueStore
public static <K,V> StoreBuilder<SessionStore<K,V>> sessionStoreBuilder(SessionBytesStoreSupplier supplier, Serde<K> keySerde, Serde<V> valueSerde)
StoreBuilder
that can be used to build a SessionStore
.K
- key typeV
- value typesupplier
- a SessionBytesStoreSupplier
(cannot be null
)keySerde
- the key serde to usevalueSerde
- the value serde to use; if the serialized bytes is null for put operations,
it is treated as deleteStoreBuilder
than can build a SessionStore