K
- Type of keysV
- Type of values@InterfaceStability.Evolving public interface KGroupedStream<K,V>
KGroupedStream
is an abstraction of a grouped record stream of KeyValue
pairs.
It is an intermediate representation of a KStream
in order to apply an aggregation operation on the original
KStream
records.
It is an intermediate representation after a grouping of a KStream
before an aggregation is applied to the
new partitions resulting in a KTable
.
A KGroupedStream
must be obtained from a KStream
via groupByKey()
or
groupBy(...)
.
KStream
Modifier and Type | Method and Description |
---|---|
<T> KTable<Windowed<K>,T> |
aggregate(Initializer<T> initializer,
Aggregator<? super K,? super V,T> aggregator,
Merger<? super K,T> sessionMerger,
SessionWindows sessionWindows,
Serde<T> aggValueSerde)
Deprecated.
|
<T> KTable<Windowed<K>,T> |
aggregate(Initializer<T> initializer,
Aggregator<? super K,? super V,T> aggregator,
Merger<? super K,T> sessionMerger,
SessionWindows sessionWindows,
Serde<T> aggValueSerde,
StateStoreSupplier<SessionStore> storeSupplier)
|
<T> KTable<Windowed<K>,T> |
aggregate(Initializer<T> initializer,
Aggregator<? super K,? super V,T> aggregator,
Merger<? super K,T> sessionMerger,
SessionWindows sessionWindows,
Serde<T> aggValueSerde,
java.lang.String queryableStoreName)
|
<VR> KTable<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<K,VR> |
aggregate(Initializer<VR> initializer,
Aggregator<? super K,? super V,VR> aggregator,
Materialized<K,VR,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Aggregate the values of records in this stream by the grouped key.
|
<VR> KTable<K,VR> |
aggregate(Initializer<VR> initializer,
Aggregator<? super K,? super V,VR> aggregator,
Serde<VR> aggValueSerde)
|
<VR> KTable<K,VR> |
aggregate(Initializer<VR> initializer,
Aggregator<? super K,? super V,VR> aggregator,
Serde<VR> aggValueSerde,
java.lang.String queryableStoreName)
|
<VR> KTable<K,VR> |
aggregate(Initializer<VR> initializer,
Aggregator<? super K,? super V,VR> aggregator,
StateStoreSupplier<KeyValueStore> storeSupplier)
|
<W extends Window,VR> |
aggregate(Initializer<VR> initializer,
Aggregator<? super K,? super V,VR> aggregator,
Windows<W> windows,
Serde<VR> aggValueSerde)
Deprecated.
use
windowedBy(windows) followed by
TimeWindowedKStream.aggregate(Initializer, Aggregator, Materialized) aggregate(initializer, aggregator, Materialized.with(null, aggValueSerde))} |
<W extends Window,VR> |
aggregate(Initializer<VR> initializer,
Aggregator<? super K,? super V,VR> aggregator,
Windows<W> windows,
Serde<VR> aggValueSerde,
java.lang.String queryableStoreName)
|
<W extends Window,VR> |
aggregate(Initializer<VR> initializer,
Aggregator<? super K,? super V,VR> aggregator,
Windows<W> windows,
StateStoreSupplier<WindowStore> storeSupplier)
Deprecated.
|
KTable<K,java.lang.Long> |
count()
Count the number of records in this stream by the grouped key.
|
KTable<K,java.lang.Long> |
count(Materialized<K,java.lang.Long,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Count the number of records in this stream by the grouped key.
|
KTable<Windowed<K>,java.lang.Long> |
count(SessionWindows sessionWindows)
Deprecated.
use
windowedBy(sessionWindows) followed by
count() |
KTable<Windowed<K>,java.lang.Long> |
count(SessionWindows sessionWindows,
StateStoreSupplier<SessionStore> storeSupplier)
Deprecated.
|
KTable<Windowed<K>,java.lang.Long> |
count(SessionWindows sessionWindows,
java.lang.String queryableStoreName)
Deprecated.
use
windowedBy(sessionWindows) followed by
count(Materialized.as(queryableStoreName)) |
KTable<K,java.lang.Long> |
count(StateStoreSupplier<KeyValueStore> storeSupplier)
Deprecated.
|
KTable<K,java.lang.Long> |
count(java.lang.String queryableStoreName)
Deprecated.
|
<W extends Window> |
count(Windows<W> windows)
Deprecated.
use
windowedBy(windows) followed by count() |
<W extends Window> |
count(Windows<W> windows,
StateStoreSupplier<WindowStore> storeSupplier)
Deprecated.
use
windowedBy(windows) followed by
count(Materialized.as(KeyValueByteStoreSupplier)) |
<W extends Window> |
count(Windows<W> windows,
java.lang.String queryableStoreName)
Deprecated.
use
windowedBy(windows) followed by
count(Materialized.as(queryableStoreName)) |
KTable<K,V> |
reduce(Reducer<V> reducer)
Combine the values of records in this stream by the grouped key.
|
KTable<K,V> |
reduce(Reducer<V> reducer,
Materialized<K,V,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Combine the value of records in this stream by the grouped key.
|
KTable<Windowed<K>,V> |
reduce(Reducer<V> reducer,
SessionWindows sessionWindows)
Deprecated.
use
windowedBy(sessionWindows) followed by
reduce(reducer) |
KTable<Windowed<K>,V> |
reduce(Reducer<V> reducer,
SessionWindows sessionWindows,
StateStoreSupplier<SessionStore> storeSupplier)
Deprecated.
|
KTable<Windowed<K>,V> |
reduce(Reducer<V> reducer,
SessionWindows sessionWindows,
java.lang.String queryableStoreName)
Deprecated.
|
KTable<K,V> |
reduce(Reducer<V> reducer,
StateStoreSupplier<KeyValueStore> storeSupplier)
Deprecated.
|
KTable<K,V> |
reduce(Reducer<V> reducer,
java.lang.String queryableStoreName)
Deprecated.
|
<W extends Window> |
reduce(Reducer<V> reducer,
Windows<W> windows)
Deprecated.
use
windowedBy(windows) followed by
reduce(reducer) |
<W extends Window> |
reduce(Reducer<V> reducer,
Windows<W> windows,
StateStoreSupplier<WindowStore> storeSupplier)
Deprecated.
|
<W extends Window> |
reduce(Reducer<V> reducer,
Windows<W> windows,
java.lang.String queryableStoreName)
Deprecated.
|
SessionWindowedKStream<K,V> |
windowedBy(SessionWindows windows)
Create a new
SessionWindowedKStream instance that can be used to perform session windowed aggregations. |
<W extends Window> |
windowedBy(Windows<W> windows)
Create a new
TimeWindowedKStream instance that can be used to perform windowed aggregations. |
@Deprecated KTable<K,java.lang.Long> count(java.lang.String queryableStoreName)
count(Materialized.as(queryableStoreName))
null
key or value are ignored.
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.
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
.
To query the local KeyValueStore
it must be obtained via
KafkaStreams#store(...)
:
KafkaStreams streams = ... // counting words
ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
String key = "some-word";
Long countForWord = localStore.get(key); // 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 must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
user-specified in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "queryableStoreName" is the
provide queryableStoreName
, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via KafkaStreams.toString()
.
queryableStoreName
- the name of the underlying KTable
state store; valid characters are ASCII
alphanumerics, '.', '_' and '-'. If null
then this will be equivalent to count()
.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<K,java.lang.Long> count()
null
key or value are ignored.
The result is written into a local KeyValueStore
(which is basically an ever-updating materialized view).
Furthermore, updates to the store are sent downstream into a KTable
changelog stream.
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()
.
KTable
that contains "update" records with unmodified keys and Long
values that
represent the latest (rolling) count (i.e., number of records) for each key@Deprecated KTable<K,java.lang.Long> count(StateStoreSupplier<KeyValueStore> storeSupplier)
count(Materialized.as(KeyValueByteStoreSupplier))
null
key or value are ignored.
The result is written into a local KeyValueStore
(which is basically an ever-updating materialized view)
provided by the given storeSupplier
.
Furthermore, updates to the store are sent downstream into a KTable
changelog stream.
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
.
To query the local KeyValueStore
it must be obtained via
KafkaStreams#store(...)
.
Use StateStore.name()
to get the store name:
KafkaStreams streams = ... // counting words
String queryableStoreName = storeSupplier.name();
ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
String key = "some-word";
Long countForWord = localStore.get(key); // 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.storeSupplier
- user defined state store supplier. Cannot be null
.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<K,java.lang.Long> count(Materialized<K,java.lang.Long,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
null
key or value are ignored.
The result is written into a local KeyValueStore
(which is basically an ever-updating materialized view)
provided by the given materialized
.
Furthermore, updates to the store are sent downstream into a KTable
changelog stream.
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
.
To query the local KeyValueStore
it must be obtained via
KafkaStreams#store(...)
.
KafkaStreams streams = ... // counting words
String queryableStoreName = "count-store"; // the queryableStoreName should be the name of the store as defined by the Materialized instance
ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
String key = "some-word";
Long countForWord = localStore.get(key); // 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@Deprecated <W extends Window> KTable<Windowed<K>,java.lang.Long> count(Windows<W> windows, java.lang.String queryableStoreName)
windowedBy(windows)
followed by
count(Materialized.as(queryableStoreName))
null
key or value are ignored.
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 provided queryableStoreName
.
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.
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
.
To query the local windowed KeyValueStore
it must be obtained via
KafkaStreams#store(...)
:
KafkaStreams streams = ... // counting words
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 must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
user-specified in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "queryableStoreName" is the
provide queryableStoreName
, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via KafkaStreams.toString()
.
windows
- the specification of the aggregation Windows
queryableStoreName
- the name of the underlying KTable
state store; valid characters are ASCII
alphanumerics, '.', '_' and '-'. If null
then this will be equivalent to count(Windows)
.KTable
that contains "update" records with unmodified keys and Long
values
that represent the latest (rolling) count (i.e., number of records) for each key within a window.@Deprecated <W extends Window> KTable<Windowed<K>,java.lang.Long> count(Windows<W> windows)
windowedBy(windows)
followed by count()
null
key or value are ignored.
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 provided queryableName
.
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.
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()
.
@Deprecated <W extends Window> KTable<Windowed<K>,java.lang.Long> count(Windows<W> windows, StateStoreSupplier<WindowStore> storeSupplier)
windowedBy(windows)
followed by
count(Materialized.as(KeyValueByteStoreSupplier))
null
key or value are ignored.
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) provided by the given storeSupplier
.
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.
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
.
To query the local windowed KeyValueStore
it must be obtained via
KafkaStreams#store(...)
.
Use StateStoreSupplier.name()
to get the store name:
KafkaStreams streams = ... // counting words
String queryableStoreName = storeSupplier.name();
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableName, 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.windows
- the specification of the aggregation Windows
storeSupplier
- user defined state store supplier. Cannot be null
.KTable
that contains "update" records with unmodified keys and Long
values
that represent the latest (rolling) count (i.e., number of records) for each key within a window@Deprecated KTable<Windowed<K>,java.lang.Long> count(SessionWindows sessionWindows, java.lang.String queryableStoreName)
windowedBy(sessionWindows)
followed by
count(Materialized.as(queryableStoreName))
SessionWindows
.
Records with null
key or value are ignored.
The result is written into a local SessionStore
(which is basically an ever-updating
materialized view) that can be queried using the provided queryableStoreName
.
SessionWindows are retained until their retention time expires (c.f. SessionWindows.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.
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
.
To query the local windowed KeyValueStore
it must be obtained via
KafkaStreams#store(...)
.
KafkaStreams streams = ... // compute sum
ReadOnlySessionStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>ReadOnlySessionStore<String, Long>);
String key = "some-key";
KeyValueIterator<Windowed<String>, Long> sumForKeyForWindows = localWindowStore.fetch(key); // 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.sessionWindows
- the specification of the aggregation SessionWindows
queryableStoreName
- the name of the state store created from this operation; valid characters are ASCII
alphanumerics, '.', '_' and '-. If null
then this will be equivalent to count(SessionWindows)
.KTable
that contains "update" records with unmodified keys and Long
values
that represent the latest (rolling) count (i.e., number of records) for each key within a window@Deprecated KTable<Windowed<K>,java.lang.Long> count(SessionWindows sessionWindows)
windowedBy(sessionWindows)
followed by
count()
SessionWindows
.
Records with null
key or value are ignored.
The result is written into a local SessionStore
(which is basically an ever-updating
materialized view) that can be queried using the provided queryableStoreName
.
SessionWindows are retained until their retention time expires (c.f. SessionWindows.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.
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
.
sessionWindows
- the specification of the aggregation SessionWindows
KTable
that contains "update" records with unmodified keys and Long
values
that represent the latest (rolling) count (i.e., number of records) for each key within a window@Deprecated KTable<Windowed<K>,java.lang.Long> count(SessionWindows sessionWindows, StateStoreSupplier<SessionStore> storeSupplier)
windowedBy(sessionWindows)
followed by
count(Materialized.as(KeyValueByteStoreSupplier))
SessionWindows
.
Records with null
key or value are ignored.
The result is written into a local SessionStore
(which is basically an ever-updating materialized view)
provided by the given storeSupplier
.
SessionWindows are retained until their retention time expires (c.f. SessionWindows.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.
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
.
To query the local windowed KeyValueStore
it must be obtained via
KafkaStreams#store(...)
.
Use StateStoreSupplier.name()
to get the store name:
KafkaStreams streams = ... // compute sum
Sting queryableStoreName = storeSupplier.name();
ReadOnlySessionStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>ReadOnlySessionStore<String, Long>);
String key = "some-key";
KeyValueIterator<Windowed<String>, Long> sumForKeyForWindows = localWindowStore.fetch(key); // 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.sessionWindows
- the specification of the aggregation SessionWindows
storeSupplier
- user defined state store supplier. Cannot be null
.KTable
that contains "update" records with unmodified keys and Long
values
that represent the latest (rolling) count (i.e., number of records) for each key within a windowKTable<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
(c.f. aggregate(Initializer, Aggregator)
).
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()
.
@Deprecated KTable<K,V> reduce(Reducer<V> reducer, java.lang.String queryableStoreName)
reduce(reducer, Materialized.as(queryableStoreName))
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
(c.f. aggregate(Initializer, Aggregator, Serde, String)
).
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 (first argument) and the record's value (second argument):
// At the example of a Reducer<Long>
new Reducer<Long>() {
public Long apply(Long aggValue, Long currValue) {
return aggValue + currValue;
}
}
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
.
To query the local KeyValueStore
it must be obtained via
KafkaStreams#store(...)
:
KafkaStreams streams = ... // compute sum
ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
String key = "some-key";
Long sumForKey = localStore.get(key); // 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 must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
user-specified in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "queryableStoreName" is the
provide queryableStoreName
, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via KafkaStreams.toString()
.
reducer
- a Reducer
that computes a new aggregate result. Cannot be null
.queryableStoreName
- the name of the underlying KTable
state store; valid characters are ASCII
alphanumerics, '.', '_' and '-'. If null
then this will be equivalent to reduce(Reducer)
()}.KTable
that contains "update" records with unmodified keys, and values that represent the
latest (rolling) aggregate for each key@Deprecated KTable<K,V> reduce(Reducer<V> reducer, StateStoreSupplier<KeyValueStore> storeSupplier)
reduce(reducer, Materialized.as(KeyValueByteStoreSupplier))
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
(c.f. aggregate(Initializer, Aggregator, org.apache.kafka.streams.processor.StateStoreSupplier)
).
The result is written into a local KeyValueStore
(which is basically an ever-updating materialized view)
provided by the given storeSupplier
.
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 (first argument) and the record's value (second argument):
// At the example of a Reducer<Long>
new Reducer<Long>() {
public Long apply(Long aggValue, Long currValue) {
return aggValue + currValue;
}
}
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, org.apache.kafka.streams.processor.StateStoreSupplier)
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
.
To query the local KeyValueStore
it must be obtained via
KafkaStreams#store(...)
.
Use StateStoreSupplier.name()
to get the store name:
KafkaStreams streams = ... // compute sum
String queryableStoreName = storeSupplier.name();
ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
String key = "some-key";
Long sumForKey = localStore.get(key); // 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.KTable<K,V> reduce(Reducer<V> reducer, Materialized<K,V,KeyValueStore<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
(c.f. aggregate(Initializer, Aggregator, Materialized)
).
The result is written into a local KeyValueStore
(which is basically an ever-updating materialized view)
provided by the given 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 (first argument) and the record's value (second argument):
// At the example of a Reducer<Long>
new Reducer<Long>() {
public Long apply(Long aggValue, Long currValue) {
return aggValue + currValue;
}
}
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, Materialized)
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
.
To query the local KeyValueStore
it must be obtained via
KafkaStreams#store(...)
.
KafkaStreams streams = ... // compute sum
String queryableStoreName = "storeName" // the queryableStoreName should be the name of the store as defined by the Materialized instance
ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
String key = "some-key";
Long sumForKey = localStore.get(key); // 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.reducer
- a Reducer
that computes a new aggregate result. Cannot be null
.materialized
- 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@Deprecated <W extends Window> KTable<Windowed<K>,V> reduce(Reducer<V> reducer, Windows<W> windows, java.lang.String queryableStoreName)
windowedBy(windows)
followed by
reduce(reducer, Materialized.as(queryableStoreName))
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
(c.f. aggregate(Initializer, Aggregator, Windows, Serde, String)
).
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 provided queryableStoreName
.
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.
The specified Reducer
is applied for each input record and computes a new aggregate using the current
aggregate (first argument) and the record's value (second argument):
// At the example of a Reducer<Long>
new Reducer<Long>() {
public Long apply(Long aggValue, Long currValue) {
return aggValue + currValue;
}
}
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, Windows, 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 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
.
To query the local windowed KeyValueStore
it must be obtained via
KafkaStreams#store(...)
:
KafkaStreams streams = ... // compute sum
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore());
String key = "some-key";
long fromTime = ...;
long toTime = ...;
WindowStoreIterator<Long> sumForKeyForWindows = 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 must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
user-specified in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "queryableStoreName" is the
provide queryableStoreName
, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via KafkaStreams.toString()
.
reducer
- a Reducer
that computes a new aggregate resultwindows
- the specification of the aggregation Windows
queryableStoreName
- the name of the state store created from this operation; valid characters are ASCII
alphanumerics, '.', '_' and '-'. If null
then this will be equivalent to reduce(Reducer, Windows)
.KTable
that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window@Deprecated <W extends Window> KTable<Windowed<K>,V> reduce(Reducer<V> reducer, Windows<W> windows)
windowedBy(windows)
followed by
reduce(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
(c.f. aggregate(Initializer, Aggregator, Windows, Serde, String)
).
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 provided queryableStoreName
.
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.
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, Windows, 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 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()
.
@Deprecated <W extends Window> KTable<Windowed<K>,V> reduce(Reducer<V> reducer, Windows<W> windows, StateStoreSupplier<WindowStore> storeSupplier)
windowedBy(windows)
followed by
reduce(reducer, Materialized.as(KeyValueByteStoreSupplier))
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
(c.f. aggregate(Initializer, Aggregator, Windows, Serde, String)
).
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) provided by the given storeSupplier
.
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.
The specified Reducer
is applied for each input record and computes a new aggregate using the current
aggregate (first argument) and the record's value (second argument):
// At the example of a Reducer<Long>
new Reducer<Long>() {
public Long apply(Long aggValue, Long currValue) {
return aggValue + currValue;
}
}
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, Windows, org.apache.kafka.streams.processor.StateStoreSupplier)
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 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
.
To query the local windowed KeyValueStore
it must be obtained via
KafkaStreams#store(...)
.
Use StateStoreSupplier.name()
to get the store name:
KafkaStreams streams = ... // compute sum
Sting queryableStoreName = storeSupplier.name();
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore());
String key = "some-key";
long fromTime = ...;
long toTime = ...;
WindowStoreIterator<Long> sumForKeyForWindows = 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.reducer
- a Reducer
that computes a new aggregate result. Cannot be null
.windows
- the specification of the aggregation Windows
storeSupplier
- user defined state store supplier. Cannot be null
.KTable
that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window@Deprecated KTable<Windowed<K>,V> reduce(Reducer<V> reducer, SessionWindows sessionWindows, java.lang.String queryableStoreName)
windowedBy(sessionWindows)
followed by
reduce(reducer, Materialized.as(queryableStoreName))
SessionWindows
.
Records with 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
(c.f. aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde, String)
).
The result is written into a local SessionStore
(which is basically an ever-updating
materialized view) that can be queried using the provided queryableStoreName
.
SessionWindows are retained until their retention time expires (c.f. SessionWindows.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.
The specified Reducer
is applied for each input record and computes a new aggregate using the current
aggregate (first argument) and the record's value (second argument):
// At the example of a Reducer<Long>
new Reducer<Long>() {
public Long apply(Long aggValue, Long currValue) {
return aggValue + currValue;
}
}
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, SessionWindows, 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 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
.
To query the local windowed KeyValueStore
it must be obtained via
KafkaStreams#store(...)
.
KafkaStreams streams = ... // compute sum
ReadOnlySessionStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>ReadOnlySessionStore<String, Long>);
String key = "some-key";
KeyValueIterator<Windowed<String>, Long> sumForKeyForWindows = localWindowStore.fetch(key); // 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 must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
user-specified in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "queryableStoreName" is the
provide queryableStoreName
, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via KafkaStreams.toString()
.
reducer
- a Reducer
that computes a new aggregate result. Cannot be null
.sessionWindows
- the specification of the aggregation SessionWindows
queryableStoreName
- the name of the state store created from this operation; valid characters are ASCII
alphanumerics, '.', '_' and '-'. If null
then this will be equivalent to reduce(Reducer, SessionWindows)
.KTable
that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window@Deprecated KTable<Windowed<K>,V> reduce(Reducer<V> reducer, SessionWindows sessionWindows)
windowedBy(sessionWindows)
followed by
reduce(reducer)
SessionWindows
.
Records with 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
(c.f. aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde, String)
).
The result is written into a local SessionStore
(which is basically an ever-updating
materialized view) that can be queried using the provided queryableStoreName
.
SessionWindows are retained until their retention time expires (c.f. SessionWindows.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.
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, SessionWindows, 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 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
.
reducer
- a Reducer
that computes a new aggregate result. Cannot be null
.sessionWindows
- the specification of the aggregation SessionWindows
KTable
that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window@Deprecated KTable<Windowed<K>,V> reduce(Reducer<V> reducer, SessionWindows sessionWindows, StateStoreSupplier<SessionStore> storeSupplier)
windowedBy(sessionWindows)
followed by
reduce(reducer, Materialized.as(KeyValueByteStoreSupplier))
SessionWindows
.
Records with 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
(c.f. aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde, String)
).
The result is written into a local SessionStore
(which is basically an ever-updating materialized view)
provided by the given storeSupplier
.
SessionWindows are retained until their retention time expires (c.f. SessionWindows.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.
The specified Reducer
is applied for each input record and computes a new aggregate using the current
aggregate (first argument) and the record's value (second argument):
// At the example of a Reducer<Long>
new Reducer<Long>() {
public Long apply(Long aggValue, Long currValue) {
return aggValue + currValue;
}
}
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, SessionWindows, org.apache.kafka.streams.processor.StateStoreSupplier)
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 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
.
To query the local windowed KeyValueStore
it must be obtained via
KafkaStreams#store(...)
.
Use StateStoreSupplier.name()
to get the store name:
KafkaStreams streams = ... // compute sum
Sting queryableStoreName = storeSupplier.name();
ReadOnlySessionStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>ReadOnlySessionStore<String, Long>);
String key = "some-key";
KeyValueIterator<Windowed<String>, Long> sumForKeyForWindows = localWindowStore.fetch(key); // 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 must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
user-specified in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "queryableStoreName" is the
provide queryableStoreName
, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via KafkaStreams.toString()
.
reducer
- a Reducer
that computes a new aggregate result. Cannot be null
.sessionWindows
- the specification of the aggregation SessionWindows
storeSupplier
- user defined state store supplier. Cannot be null
.KTable
that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window@Deprecated <VR> KTable<K,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Serde<VR> aggValueSerde, java.lang.String queryableStoreName)
aggregate(initializer, aggregator, Materialized.as(queryableStoreName).withValueSerde(aggValueSerde))
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, Serde, String)
can be used to compute aggregate functions like
count (c.f. count(String)
).
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
.
To query the local KeyValueStore
it must be obtained via
KafkaStreams#store(...)
:
KafkaStreams streams = ... // some aggregation on value type double
ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
String key = "some-key";
Long aggForKey = localStore.get(key); // 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 must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
user-specified in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "queryableStoreName" is the
provide queryableStoreName
, and "-changelog" is a fixed suffix.
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 resultaggValueSerde
- aggregate value serdes for materializing the aggregated table,
if not specified the default serdes defined in the configs will be usedqueryableStoreName
- the name of the state store created from this operation; valid characters are ASCII
alphanumerics, '.', '_' and '-'. If null
then this will be equivalent to aggregate(Initializer, Aggregator, Serde)
.KTable
that contains "update" records with unmodified keys, and values that represent the
latest (rolling) aggregate for each key<VR> KTable<K,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Materialized<K,VR,KeyValueStore<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 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, Serde, String)
can be used to compute aggregate functions like
count (c.f. count()
).
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
.
To query the local KeyValueStore
it must be obtained via
KafkaStreams#store(...)
:
KafkaStreams streams = ... // some aggregation on value type double
ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
String key = "some-key";
Long aggForKey = localStore.get(key); // 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 must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
user-specified in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "queryableStoreName" is the
provide queryableStoreName
, and "-changelog" is a fixed suffix.
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 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<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@Deprecated <VR> KTable<K,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Serde<VR> aggValueSerde)
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, Serde, String)
can be used to compute aggregate functions like
count (c.f. count()
).
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 resultaggValueSerde
- aggregate value serdes for materializing the aggregated table,
if not specified the default serdes defined in the configs will be usedKTable
that contains "update" records with unmodified keys, and values that represent the
latest (rolling) aggregate for each key@Deprecated <VR> KTable<K,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, StateStoreSupplier<KeyValueStore> storeSupplier)
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)
provided by the given storeSupplier
.
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, org.apache.kafka.streams.processor.StateStoreSupplier)
can be
used to compute aggregate functions like count (c.f. count()
).
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
.
To query the local KeyValueStore
it must be obtained via
KafkaStreams#store(...)
.
Use StateStoreSupplier.name()
to get the store name:
KafkaStreams streams = ... // some aggregation on value type double
Sting queryableStoreName = storeSupplier.name();
ReadOnlyKeyValueStore<String,Long> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>keyValueStore());
String key = "some-key";
Long aggForKey = localStore.get(key); // 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.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 resultstoreSupplier
- user defined state store supplier. Cannot be null
.KTable
that contains "update" records with unmodified keys, and values that represent the
latest (rolling) aggregate for each key@Deprecated <W extends Window,VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Windows<W> windows, Serde<VR> aggValueSerde, java.lang.String queryableStoreName)
windowedBy(windows)
followed by
aggregate(initializer, aggregator, Materialized.as(queryableStoreName).withValueSerde(aggValueSerde))
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 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 provided queryableStoreName
.
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.
The specified Initializer
is applied once per window 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, Windows, Serde, String)
can be used to compute aggregate
functions like count (c.f. count(Windows)
).
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
.
To query the local windowed KeyValueStore
it must be obtained via
KafkaStreams#store(...)
:
KafkaStreams streams = ... // some windowed aggregation on value type double
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore());
String key = "some-key";
long fromTime = ...;
long toTime = ...;
WindowStoreIterator<Long> aggForKeyForWindows = 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 must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog", where "applicationId" is
user-specified in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "queryableStoreName" is the
provide queryableStoreName
, and "-changelog" is a fixed suffix.
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 resultwindows
- the specification of the aggregation Windows
aggValueSerde
- aggregate value serdes for materializing the aggregated table,
if not specified the default serdes defined in the configs will be usedqueryableStoreName
- the name of the state store created from this operation; valid characters are ASCII
alphanumerics, '.', '_' and '-'. If null
then this will be equivalent to aggregate(Initializer, Aggregator, Windows, Serde)
.KTable
that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window@Deprecated <W extends Window,VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Windows<W> windows, Serde<VR> aggValueSerde)
windowedBy(windows)
followed by
TimeWindowedKStream.aggregate(Initializer, Aggregator, Materialized)
aggregate(initializer, aggregator, Materialized.with(null, aggValueSerde))}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 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 provided queryableStoreName
.
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.
The specified Initializer
is applied once per window 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, Windows, Serde, String)
can be used to compute aggregate
functions like count (c.f. count(Windows)
).
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()
.
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 resultwindows
- the specification of the aggregation Windows
aggValueSerde
- aggregate value serdes for materializing the aggregated table,
if not specified the default serdes defined in the configs will be usedKTable
that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window@Deprecated <W extends Window,VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Windows<W> windows, StateStoreSupplier<WindowStore> storeSupplier)
windowedBy(windows)
followed by
aggregate(initializer, aggregator, Materialized.as(KeyValueByteStoreSupplier))
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 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) provided by the given storeSupplier
.
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.
The specified Initializer
is applied once per window 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, Windows, org.apache.kafka.streams.processor.StateStoreSupplier)
can be used to compute aggregate functions like count (c.f. count(Windows)
).
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
.
To query the local windowed KeyValueStore
it must be obtained via
KafkaStreams#store(...)
.
Use StateStoreSupplier.name()
to get the store name:
KafkaStreams streams = ... // some windowed aggregation on value type Long
Sting queryableStoreName = storeSupplier.name();
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore());
String key = "some-key";
long fromTime = ...;
long toTime = ...;
WindowStoreIterator<Long> aggForKeyForWindows = 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.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 resultwindows
- the specification of the aggregation Windows
storeSupplier
- user defined state store supplier. Cannot be null
.KTable
that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window@Deprecated <T> KTable<Windowed<K>,T> aggregate(Initializer<T> initializer, Aggregator<? super K,? super V,T> aggregator, Merger<? super K,T> sessionMerger, SessionWindows sessionWindows, Serde<T> aggValueSerde, java.lang.String queryableStoreName)
windowedBy(sessionWindows)
followed by
aggregate(initializer, aggregator, sessionMerger, Materialized.as(queryableStoreName).withValueSerde(aggValueSerde))
SessionWindows
.
Records with 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 SessionStore
(which is basically an ever-updating
materialized view) that can be queried using the provided queryableStoreName
.
SessionWindows are retained until their retention time expires (c.f. SessionWindows.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.
The specified Initializer
is applied once per session 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, Merger, SessionWindows, Serde, String)
can be used to compute
aggregate functions like count (c.f. count(SessionWindows)
)
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
.
To query the local SessionStore
it must be obtained via
KafkaStreams#store(...)
.
KafkaStreams streams = ... // some windowed aggregation on value type double
ReadOnlySessionStore<String, Long> sessionStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>sessionStore());
String key = "some-key";
KeyValueIterator<Windowed<String>, Long> aggForKeyForSession = localWindowStore.fetch(key); // 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.T
- the value type of the resulting KTable
initializer
- the instance of Initializer
aggregator
- the instance of Aggregator
sessionMerger
- the instance of Merger
sessionWindows
- the specification of the aggregation SessionWindows
aggValueSerde
- aggregate value serdes for materializing the aggregated table,
if not specified the default serdes defined in the configs will be usedqueryableStoreName
- the name of the state store created from this operation; valid characters are ASCII
alphanumerics, '.', '_' and '-'. If null
then this will be equivalent to aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde)
.KTable
that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window@Deprecated <T> KTable<Windowed<K>,T> aggregate(Initializer<T> initializer, Aggregator<? super K,? super V,T> aggregator, Merger<? super K,T> sessionMerger, SessionWindows sessionWindows, Serde<T> aggValueSerde)
windowedBy(sessionWindows)
followed by
aggregate(initializer, aggregator, sessionMerger, Materialized.with(null, aggValueSerde))
SessionWindows
.
Records with 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 SessionStore
(which is basically an ever-updating
materialized view) that can be queried using the provided queryableStoreName
.
SessionWindows are retained until their retention time expires (c.f. SessionWindows.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.
The specified Initializer
is applied once per session 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, Merger, SessionWindows, Serde, String)
can be used to compute
aggregate functions like count (c.f. count(SessionWindows)
)
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
.
T
- the value type of the resulting KTable
initializer
- the instance of Initializer
aggregator
- the instance of Aggregator
sessionMerger
- the instance of Merger
sessionWindows
- the specification of the aggregation SessionWindows
aggValueSerde
- aggregate value serdes for materializing the aggregated table,
if not specified the default serdes defined in the configs will be usedKTable
that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window@Deprecated <T> KTable<Windowed<K>,T> aggregate(Initializer<T> initializer, Aggregator<? super K,? super V,T> aggregator, Merger<? super K,T> sessionMerger, SessionWindows sessionWindows, Serde<T> aggValueSerde, StateStoreSupplier<SessionStore> storeSupplier)
windowedBy(sessionWindows)
followed by
aggregate(initializer, aggregator, sessionMerger, Materialized.as(KeyValueByteStoreSupplier).withValueSerde(aggValueSerde))
SessionWindows
.
Records with 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 SessionStore
(which is basically an ever-updating materialized view)
provided by the given storeSupplier
.
SessionWindows are retained until their retention time expires (c.f. SessionWindows.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.
The specified Initializer
is applied once per session 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, Merger, SessionWindows, Serde, org.apache.kafka.streams.processor.StateStoreSupplier)
can be used to compute aggregate functions like count (c.f. count(SessionWindows)
).
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
.
To query the local SessionStore
it must be obtained via
KafkaStreams#store(...)
.
Use StateStoreSupplier.name()
to get the store name:
KafkaStreams streams = ... // some windowed aggregation on value type double
Sting queryableStoreName = storeSupplier.name();
ReadOnlySessionStore<String, Long> sessionStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>sessionStore());
String key = "some-key";
KeyValueIterator<Windowed<String>, Long> aggForKeyForSession = localWindowStore.fetch(key); // 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.T
- the value type of the resulting KTable
initializer
- the instance of Initializer
aggregator
- the instance of Aggregator
sessionMerger
- the instance of Merger
sessionWindows
- the specification of the aggregation SessionWindows
aggValueSerde
- aggregate value serdes for materializing the aggregated table,
if not specified the default serdes defined in the configs will be usedstoreSupplier
- user defined state store supplier. Cannot be null
.KTable
that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window<W extends Window> TimeWindowedKStream<K,V> windowedBy(Windows<W> windows)
TimeWindowedKStream
instance that can be used to perform windowed aggregations.W
- the window typewindows
- the specification of the aggregation Windows
TimeWindowedKStream
SessionWindowedKStream<K,V> windowedBy(SessionWindows windows)
SessionWindowedKStream
instance that can be used to perform session windowed aggregations.windows
- the specification of the aggregation SessionWindows
TimeWindowedKStream