K - Type of keysV - Type of valuespublic interface TimeWindowedKStream<K,V>
TimeWindowedKStream is an abstraction of a windowed record stream of KeyValue pairs.
It is an intermediate representation of a KStream in order to apply a windowed aggregation operation on the original
KStream records.
It is an intermediate representation after a grouping and windowing of a KStream before an aggregation is applied to the
new (partitioned) windows resulting in a windowed KTable
(a KTable is a KTable with key type Windowed.
The specified windows define either hopping time windows that can be overlapping or tumbling (c.f.
TimeWindows) or they define landmark windows (c.f. UnlimitedWindows).
The result is written into a local windowed KeyValueStore (which is basically an ever-updating
materialized view) that can be queried using the name provided in the Materialized instance.
New events are added to windows until their grace period ends (see TimeWindows.grace(Duration)).
Furthermore, updates to the store are sent downstream into a windowed KTable changelog stream, where
"windowed" implies that the KTable key is a combined key of the original record key and a window ID.
A WindowedKStream must be obtained from a KGroupedStream via KGroupedStream.windowedBy(Windows) .
KStream,
KGroupedStream| Modifier and Type | Method and Description |
|---|---|
<VR> KTable<Windowed<K>,VR> |
aggregate(Initializer<VR> initializer,
Aggregator<? super K,? super V,VR> aggregator)
Aggregate the values of records in this stream by the grouped key.
|
<VR> KTable<Windowed<K>,VR> |
aggregate(Initializer<VR> initializer,
Aggregator<? super K,? super V,VR> aggregator,
Materialized<K,VR,WindowStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Aggregate the values of records in this stream by the grouped key.
|
<VR> KTable<Windowed<K>,VR> |
aggregate(Initializer<VR> initializer,
Aggregator<? super K,? super V,VR> aggregator,
Named named)
Aggregate the values of records in this stream by the grouped key.
|
<VR> KTable<Windowed<K>,VR> |
aggregate(Initializer<VR> initializer,
Aggregator<? super K,? super V,VR> aggregator,
Named named,
Materialized<K,VR,WindowStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Aggregate the values of records in this stream by the grouped key.
|
KTable<Windowed<K>,Long> |
count()
Count the number of records in this stream by the grouped key and the defined windows.
|
KTable<Windowed<K>,Long> |
count(Materialized<K,Long,WindowStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Count the number of records in this stream by the grouped key and the defined windows.
|
KTable<Windowed<K>,Long> |
count(Named named)
Count the number of records in this stream by the grouped key and the defined windows.
|
KTable<Windowed<K>,Long> |
count(Named named,
Materialized<K,Long,WindowStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Count the number of records in this stream by the grouped key and the defined windows.
|
KTable<Windowed<K>,V> |
reduce(Reducer<V> reducer)
Combine the values of records in this stream by the grouped key.
|
KTable<Windowed<K>,V> |
reduce(Reducer<V> reducer,
Materialized<K,V,WindowStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Combine the values of records in this stream by the grouped key.
|
KTable<Windowed<K>,V> |
reduce(Reducer<V> reducer,
Named named)
Combine the values of records in this stream by the grouped key.
|
KTable<Windowed<K>,V> |
reduce(Reducer<V> reducer,
Named named,
Materialized<K,V,WindowStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Combine the values of records in this stream by the grouped key.
|
KTable<Windowed<K>,Long> count()
null key or value are ignored.
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same window and key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "internalStoreName" is an internal name
and "-changelog" is a fixed suffix.
Note that the internal store name may not be queriable through Interactive Queries.
You can retrieve all generated internal topic names via Topology.describe().
KTable<Windowed<K>,Long> count(Named named)
null key or value are ignored.
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same window and key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "internalStoreName" is an internal name
and "-changelog" is a fixed suffix.
Note that the internal store name may not be queriable through Interactive Queries.
You can retrieve all generated internal topic names via Topology.describe().
KTable<Windowed<K>,Long> count(Materialized<K,Long,WindowStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
null key or value are ignored.
Not all updates might get sent downstream, as an internal cache will be used to deduplicate consecutive updates to
the same window and key if caching is enabled on the Materialized instance.
When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall
To query the local windowed KeyValueStore it must be obtained via
KafkaStreams#store(...):
KafkaStreams streams = ... // counting words
Store queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore());
String key = "some-word";
long fromTime = ...;
long toTime = ...;
WindowStoreIterator<Long> countForWordsForWindows = localWindowStore.fetch(key, timeFrom, timeTo); // key must be local (application state is shared over all running Kafka Streams instances)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
Therefore, the store name defined by the Materialized instance must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "storeName" is the
provide store name defined in Materialized, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via Topology.describe().
materialized - an instance of Materialized used to materialize a state store. Cannot be null.
Note: the valueSerde will be automatically set to Serdes#Long()
if there is no valueSerde providedKTable that contains "update" records with unmodified keys and Long values that
represent the latest (rolling) count (i.e., number of records) for each keyKTable<Windowed<K>,Long> count(Named named, Materialized<K,Long,WindowStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
null key or value are ignored.
Not all updates might get sent downstream, as an internal cache will be used to deduplicate consecutive updates to
the same window and key if caching is enabled on the Materialized instance.
When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall
To query the local windowed KeyValueStore it must be obtained via
KafkaStreams#store(...):
KafkaStreams streams = ... // counting words
Store queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore());
String key = "some-word";
long fromTime = ...;
long toTime = ...;
WindowStoreIterator<Long> countForWordsForWindows = localWindowStore.fetch(key, timeFrom, timeTo); // key must be local (application state is shared over all running Kafka Streams instances)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
Therefore, the store name defined by the Materialized instance must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "storeName" is the
provide store name defined in Materialized, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via Topology.describe().
named - a Named config used to name the processor in the topologymaterialized - an instance of Materialized used to materialize a state store. Cannot be null.
Note: the valueSerde will be automatically set to Serdes#Long()
if there is no valueSerde providedKTable that contains "update" records with unmodified keys and Long values that
represent the latest (rolling) count (i.e., number of records) for each key<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator)
null key or value are ignored.
Aggregating is a generalization of combining via reduce(...) as it, for example,
allows the result to have a different type than the input values.
The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
that can be queried using the provided queryableStoreName.
Furthermore, updates to the store are sent downstream into a KTable changelog stream.
The specified Initializer is applied once directly before the first input record is processed to
provide an initial intermediate aggregation result that is used to process the first record.
The specified Aggregator is applied for each input record and computes a new aggregate using the current
aggregate (or for the very first record using the intermediate aggregation result provided via the
Initializer) and the record's value.
Thus, aggregate(Initializer, Aggregator) can be used to compute aggregate functions like
count (c.f. count()).
The default value serde from config will be used for serializing the result.
If a different serde is required then you should use aggregate(Initializer, Aggregator, Materialized).
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "internalStoreName" is an internal name
and "-changelog" is a fixed suffix.
Note that the internal store name may not be queriable through Interactive Queries.
You can retrieve all generated internal topic names via Topology.describe().
VR - the value type of the resulting KTableinitializer - an Initializer that computes an initial intermediate aggregation resultaggregator - an Aggregator that computes a new aggregate resultKTable that contains "update" records with unmodified keys, and values that represent the
latest (rolling) aggregate for each key<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Named named)
null key or value are ignored.
Aggregating is a generalization of combining via reduce(...) as it, for example,
allows the result to have a different type than the input values.
The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
that can be queried using the provided queryableStoreName.
Furthermore, updates to the store are sent downstream into a KTable changelog stream.
The specified Initializer is applied once directly before the first input record is processed to
provide an initial intermediate aggregation result that is used to process the first record.
The specified Aggregator is applied for each input record and computes a new aggregate using the current
aggregate (or for the very first record using the intermediate aggregation result provided via the
Initializer) and the record's value.
Thus, aggregate(Initializer, Aggregator) can be used to compute aggregate functions like
count (c.f. count()).
The default value serde from config will be used for serializing the result.
If a different serde is required then you should use aggregate(Initializer, Aggregator, Materialized).
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "internalStoreName" is an internal name
and "-changelog" is a fixed suffix.
Note that the internal store name may not be queriable through Interactive Queries.
You can retrieve all generated internal topic names via Topology.describe().
VR - the value type of the resulting KTableinitializer - an Initializer that computes an initial intermediate aggregation resultaggregator - an Aggregator that computes a new aggregate resultnamed - a Named config used to name the processor in the topologyKTable that contains "update" records with unmodified keys, and values that represent the
latest (rolling) aggregate for each key<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Materialized<K,VR,WindowStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
null key or value are ignored.
Aggregating is a generalization of combining via reduce(...) as it, for example,
allows the result to have a different type than the input values.
The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
that can be queried using the store name as provided with Materialized.
The specified Initializer is applied once directly before the first input record is processed to
provide an initial intermediate aggregation result that is used to process the first record.
The specified Aggregator is applied for each input record and computes a new aggregate using the current
aggregate (or for the very first record using the intermediate aggregation result provided via the
Initializer) and the record's value.
Thus, aggregate(Initializer, Aggregator, Materialized) can be used to compute aggregate functions like
count (c.f. count()).
Not all updates might get sent downstream, as an internal cache will be used to deduplicate consecutive updates to
the same window and key if caching is enabled on the Materialized instance.
When caching is enable the rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall
To query the local windowed KeyValueStore it must be obtained via
KafkaStreams#store(...):
KafkaStreams streams = ... // counting words
Store queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore());
String key = "some-word";
long fromTime = ...;
long toTime = ...;
WindowStoreIterator<Long> aggregateStore = localWindowStore.fetch(key, timeFrom, timeTo); // key must be local (application state is shared over all running Kafka Streams instances)
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
Therefore, the store name defined by the Materialized instance must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "storeName" is the
provide store name defined in Materialized, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via Topology.describe().
VR - the value type of the resulting KTableinitializer - an Initializer that computes an initial intermediate aggregation resultaggregator - an Aggregator that computes a new aggregate resultmaterialized - an instance of Materialized used to materialize a state store. Cannot be null.KTable that contains "update" records with unmodified keys, and values that represent the
latest (rolling) aggregate for each key<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Named named, Materialized<K,VR,WindowStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
null key or value are ignored.
Aggregating is a generalization of combining via reduce(...) as it, for example,
allows the result to have a different type than the input values.
The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
that can be queried using the store name as provided with Materialized.
The specified Initializer is applied once directly before the first input record is processed to
provide an initial intermediate aggregation result that is used to process the first record.
The specified Aggregator is applied for each input record and computes a new aggregate using the current
aggregate (or for the very first record using the intermediate aggregation result provided via the
Initializer) and the record's value.
Thus, aggregate(Initializer, Aggregator, Materialized) can be used to compute aggregate functions like
count (c.f. count()).
Not all updates might get sent downstream, as an internal cache will be used to deduplicate consecutive updates to
the same window and key if caching is enabled on the Materialized instance.
When caching is enable the rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall
To query the local windowed KeyValueStore it must be obtained via
KafkaStreams#store(...):
KafkaStreams streams = ... // counting words
Store queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore());
String key = "some-word";
long fromTime = ...;
long toTime = ...;
WindowStoreIterator<Long> aggregateStore = localWindowStore.fetch(key, timeFrom, timeTo); // key must be local (application state is shared over all running Kafka Streams instances)
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
Therefore, the store name defined by the Materialized instance must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "storeName" is the
provide store name defined in Materialized, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via Topology.describe().
VR - the value type of the resulting KTableinitializer - an Initializer that computes an initial intermediate aggregation resultaggregator - an Aggregator that computes a new aggregate resultnamed - a Named config used to name the processor in the topologymaterialized - an instance of Materialized used to materialize a state store. Cannot be null.KTable that contains "update" records with unmodified keys, and values that represent the
latest (rolling) aggregate for each keyKTable<Windowed<K>,V> reduce(Reducer<V> reducer)
null key or value are ignored.
Combining implies that the type of the aggregate result is the same as the type of the input value.
The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
that can be queried using the provided queryableStoreName.
Furthermore, updates to the store are sent downstream into a KTable changelog stream.
The specified Reducer is applied for each input record and computes a new aggregate using the current
aggregate and the record's value.
If there is no current aggregate the Reducer is not applied and the new aggregate will be the record's
value as-is.
Thus, reduce(Reducer, String) can be used to compute aggregate functions like sum, min, or max.
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "internalStoreName" is an internal name
and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via Topology.describe().
KTable<Windowed<K>,V> reduce(Reducer<V> reducer, Named named)
null key or value are ignored.
Combining implies that the type of the aggregate result is the same as the type of the input value.
The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
that can be queried using the provided queryableStoreName.
Furthermore, updates to the store are sent downstream into a KTable changelog stream.
The specified Reducer is applied for each input record and computes a new aggregate using the current
aggregate and the record's value.
If there is no current aggregate the Reducer is not applied and the new aggregate will be the record's
value as-is.
Thus, reduce(Reducer, String) can be used to compute aggregate functions like sum, min, or max.
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "internalStoreName" is an internal name
and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via Topology.describe().
KTable<Windowed<K>,V> reduce(Reducer<V> reducer, Materialized<K,V,WindowStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
null key or value are ignored.
Combining implies that the type of the aggregate result is the same as the type of the input value.
The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
that can be queried using the store name as provided with Materialized.
Furthermore, updates to the store are sent downstream into a KTable changelog stream.
The specified Reducer is applied for each input record and computes a new aggregate using the current
aggregate and the record's value.
If there is no current aggregate the Reducer is not applied and the new aggregate will be the record's
value as-is.
Thus, reduce(Reducer, String) can be used to compute aggregate functions like sum, min, or max.
Not all updates might get sent downstream, as an internal cache will be used to deduplicate consecutive updates to
the same window and key if caching is enabled on the Materialized instance.
When caching is enable the rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall
To query the local windowed KeyValueStore it must be obtained via
KafkaStreams#store(...):
KafkaStreams streams = ... // counting words
Store queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore());
String key = "some-word";
long fromTime = ...;
long toTime = ...;
WindowStoreIterator<Long> reduceStore = localWindowStore.fetch(key, timeFrom, timeTo); // key must be local (application state is shared over all running Kafka Streams instances)
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
Therefore, the store name defined by the Materialized instance must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "storeName" is the
provide store name defined in Materialized, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via Topology.describe().
reducer - a Reducer that computes a new aggregate resultmaterialized - an instance of Materialized used to materialize a state store. Cannot be null.KTable that contains "update" records with unmodified keys, and values that represent the
latest (rolling) aggregate for each keyKTable<Windowed<K>,V> reduce(Reducer<V> reducer, Named named, Materialized<K,V,WindowStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
null key or value are ignored.
Combining implies that the type of the aggregate result is the same as the type of the input value.
The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
that can be queried using the store name as provided with Materialized.
Furthermore, updates to the store are sent downstream into a KTable changelog stream.
The specified Reducer is applied for each input record and computes a new aggregate using the current
aggregate and the record's value.
If there is no current aggregate the Reducer is not applied and the new aggregate will be the record's
value as-is.
Thus, reduce(Reducer, String) can be used to compute aggregate functions like sum, min, or max.
Not all updates might get sent downstream, as an internal cache will be used to deduplicate consecutive updates to
the same window and key if caching is enabled on the Materialized instance.
When caching is enable the rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall
To query the local windowed KeyValueStore it must be obtained via
KafkaStreams#store(...):
KafkaStreams streams = ... // counting words
Store queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore());
String key = "some-word";
long fromTime = ...;
long toTime = ...;
WindowStoreIterator<Long> reduceStore = localWindowStore.fetch(key, timeFrom, timeTo); // key must be local (application state is shared over all running Kafka Streams instances)
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
Therefore, the store name defined by the Materialized instance must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "storeName" is the
provide store name defined in Materialized, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via Topology.describe().
reducer - a Reducer that computes a new aggregate resultnamed - a Named config used to name the processor in the topologymaterialized - an instance of Materialized used to materialize a state store. Cannot be null.KTable that contains "update" records with unmodified keys, and values that represent the
latest (rolling) aggregate for each key