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.
Windows are retained until their retention time expires (c.f. Windows.until(long)
).
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 | 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.
|
KTable<Windowed<K>,java.lang.Long> |
count() |
Count the number of records in this stream by the grouped key and the defined windows.
|
KTable<Windowed<K>,java.lang.Long> |
count(Materialized<K,java.lang.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>,java.lang.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 KafkaStreams.toString()
.
KTable
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>,java.lang.Long> count(Materialized<K,java.lang.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.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 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 KafkaStreams.toString()
.
VR
- the value type of the resulting KTable
initializer
- 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, 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)
VR
- the value type of the resulting KTable
initializer
- 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 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 KafkaStreams.toString()
.
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)