Interface SessionWindowedKStream<K,V>
- Type Parameters:
K
- Type of keysV
- Type of values
SessionWindowedKStream
is an abstraction of a windowed record stream of KeyValue
pairs.
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
).
SessionWindows
are dynamic data driven windows.
They have no fixed time boundaries, rather the size of the window is determined by the records.
The result is written into a local SessionStore
(which is basically an ever-updating
materialized view) that can be queried using the name provided in the Materialized
instance.
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.
New events are added to sessions until their grace period ends (see SessionWindows.ofInactivityGapAndGrace(Duration, Duration)
).
A SessionWindowedKStream
must be obtained from a KGroupedStream
via
KGroupedStream.windowedBy(SessionWindows)
.
- See Also:
-
Method Summary
Modifier and TypeMethodDescriptionaggregate
(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Merger<? super K, VR> sessionMerger) Aggregate the values of records in this stream by the grouped key and defined sessions.aggregate
(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Merger<? super K, VR> sessionMerger, Materialized<K, VR, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in this stream by the grouped key and defined sessions.aggregate
(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Merger<? super K, VR> sessionMerger, Named named) Aggregate the values of records in this stream by the grouped key and defined sessions.aggregate
(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Merger<? super K, VR> sessionMerger, Named named, Materialized<K, VR, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in this stream by the grouped key and defined sessions.count()
Count the number of records in this stream by the grouped key and defined sessions.count
(Materialized<K, Long, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Count the number of records in this stream by the grouped key and defined sessions.Count the number of records in this stream by the grouped key and defined sessions.count
(Named named, Materialized<K, Long, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Count the number of records in this stream by the grouped key and defined sessions.emitStrategy
(EmitStrategy emitStrategy) Configure when the aggregated result will be emitted forSessionWindowedKStream
.Combine the values of records in this stream by the grouped key and defined sessions.reduce
(Reducer<V> reducer, Materialized<K, V, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Combine the values of records in this stream by the grouped key and defined sessions.Combine the values of records in this stream by the grouped key and defined sessions.reduce
(Reducer<V> reducer, Named named, Materialized<K, V, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Combine the values of records in this stream by the grouped key and defined sessions.
-
Method Details
-
count
Count the number of records in this stream by the grouped key and defined sessions. Note that sessions are generated on a per-key basis and records with different keys create independent sessions. Records withnull
key or value are ignored.The result is written into a local
SessionStore
(which is basically an ever-updating materialized view). The default key serde from the config will be used for serializing the result. If a different serde is required then you should usecount(Materialized)
. Furthermore, updates to the store are sent downstream into aKTable
changelog stream. Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to the same session 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 theconfiguration
parameters forcache size
, andcommit interval
.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 parameterAPPLICATION_ID_CONFIG
, "internalStoreName" is an internal name and "-changelog" is a fixed suffix. Note that the internal store name may not be queryable through Interactive Queries.You can retrieve all generated internal topic names via
Topology.describe()
. -
count
Count the number of records in this stream by the grouped key and defined sessions. Note that sessions are generated on a per-key basis and records with different keys create independent sessions. Records withnull
key or value are ignored.The result is written into a local
SessionStore
(which is basically an ever-updating materialized view). The default key serde from the config will be used for serializing the result. If a different serde is required then you should usecount(Named, Materialized)
. Furthermore, updates to the store are sent downstream into aKTable
changelog stream. Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to the same session 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 theconfiguration
parameters forcache size
, andcommit interval
.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 parameterAPPLICATION_ID_CONFIG
, "internalStoreName" is an internal name and "-changelog" is a fixed suffix. Note that the internal store name may not be queryable through Interactive Queries.You can retrieve all generated internal topic names via
Topology.describe()
. -
count
KTable<Windowed<K>,Long> count(Materialized<K, Long, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Count the number of records in this stream by the grouped key and defined sessions. Note that sessions are generated on a per-key basis and records with different keys create independent sessions. Records withnull
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 name provided withMaterialized
. Furthermore, updates to the store are sent downstream into aKTable
changelog stream.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 theconfiguration
parameters forcache size
, andcommit interval
.To query the local
SessionStore
it must be obtained viaKafkaStreams#store(...)
:
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // compute sum Sting queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance 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)
KafkaStreams.metadataForAllStreamsClients()
to query the value of the key on a parallel running instance of your Kafka Streams application.For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. Therefore, the store name defined by the Materialized instance must be a valid Kafka topic name and cannot contain characters other than ASCII alphanumerics, '.', '_' and '-'. The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is user-specified in
StreamsConfig
via parameterAPPLICATION_ID_CONFIG
, "storeName" is the provide store name defined inMaterialized
, and "-changelog" is a fixed suffix.You can retrieve all generated internal topic names via
Topology.describe()
.- Parameters:
materialized
- an instance ofMaterialized
used to materialize a state store. Cannot benull
. Note: the valueSerde will be automatically set toSerdes#Long()
if there is no valueSerde provided- Returns:
- a windowed
KTable
that contains "update" records with unmodified keys andLong
values that represent the latest (rolling) count (i.e., number of records) for each key per session
-
count
KTable<Windowed<K>,Long> count(Named named, Materialized<K, Long, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Count the number of records in this stream by the grouped key and defined sessions. Note that sessions are generated on a per-key basis and records with different keys create independent sessions. Records withnull
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 name provided withMaterialized
. Furthermore, updates to the store are sent downstream into aKTable
changelog stream.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 theconfiguration
parameters forcache size
, andcommit interval
.To query the local
SessionStore
it must be obtained viaKafkaStreams#store(...)
:
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // compute sum Sting queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance 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)
KafkaStreams.metadataForAllStreamsClients()
to query the value of the key on a parallel running instance of your Kafka Streams application.For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. Therefore, the store name defined by the Materialized instance must be a valid Kafka topic name and cannot contain characters other than ASCII alphanumerics, '.', '_' and '-'. The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is user-specified in
StreamsConfig
via parameterAPPLICATION_ID_CONFIG
, "storeName" is the provide store name defined inMaterialized
, and "-changelog" is a fixed suffix.You can retrieve all generated internal topic names via
Topology.describe()
.- Parameters:
named
- aNamed
config used to name the processor in the topology. Cannot benull
.materialized
- an instance ofMaterialized
used to materialize a state store. Cannot benull
. Note: the valueSerde will be automatically set toSerdes#Long()
if there is no valueSerde provided- Returns:
- a windowed
KTable
that contains "update" records with unmodified keys andLong
values that represent the latest (rolling) count (i.e., number of records) for each key per session
-
aggregate
<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Merger<? super K, VR> sessionMerger) Aggregate the values of records in this stream by the grouped key and defined sessions. Note that sessions are generated on a per-key basis and records with different keys create independent sessions. Records withnull
key or value are ignored. Aggregating is a generalization ofcombining via reduce(...)
as it, for example, allows the result to have a different type than the input values. The result is written into a localSessionStore
(which is basically an ever-updating materialized view). Furthermore, updates to the store are sent downstream into aKTable
changelog stream.The specified
Initializer
is applied directly before the first input record per session is processed to provide an initial intermediate aggregation result that is used to process the first record per session. The specifiedAggregator
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 theInitializer
) and the record's value. The specifiedMerger
is used to merge two existing sessions into one, i.e., when the windows overlap, they are merged into a single session and the old sessions are discarded. Thus,aggregate()
can be used to compute aggregate functions like count (c.f.count()
).The default key and value serde from the config will be used for serializing the result. If a different serde is required then you should use
aggregate(Initializer, Aggregator, Merger, Materialized)
.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 forcache size
, andcommit interval
.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 parameterAPPLICATION_ID_CONFIG
, "internalStoreName" is an internal name and "-changelog" is a fixed suffix. Note that the internal store name may not be queryable through Interactive Queries.You can retrieve all generated internal topic names via
Topology.describe()
.- Type Parameters:
VR
- the value type of the resultingKTable
- Parameters:
initializer
- anInitializer
that computes an initial intermediate aggregation result. Cannot benull
.aggregator
- anAggregator
that computes a new aggregate result. Cannot benull
.sessionMerger
- aMerger
that combines two aggregation results. Cannot benull
.- Returns:
- a windowed
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key per session
-
aggregate
<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Merger<? super K, VR> sessionMerger, Named named) Aggregate the values of records in this stream by the grouped key and defined sessions. Note that sessions are generated on a per-key basis and records with different keys create independent sessions. Records withnull
key or value are ignored. Aggregating is a generalization ofcombining via reduce(...)
as it, for example, allows the result to have a different type than the input values. The result is written into a localSessionStore
(which is basically an ever-updating materialized view). Furthermore, updates to the store are sent downstream into aKTable
changelog stream.The specified
Initializer
is applied directly before the first input record per session is processed to provide an initial intermediate aggregation result that is used to process the first record per session. The specifiedAggregator
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 theInitializer
) and the record's value. The specifiedMerger
is used to merge two existing sessions into one, i.e., when the windows overlap, they are merged into a single session and the old sessions are discarded. Thus,aggregate()
can be used to compute aggregate functions like count (c.f.count()
).The default key and value serde from the config will be used for serializing the result. If a different serde is required then you should use
aggregate(Initializer, Aggregator, Merger, Named, Materialized)
.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 forcache size
, andcommit interval
.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 parameterAPPLICATION_ID_CONFIG
, "internalStoreName" is an internal name and "-changelog" is a fixed suffix. Note that the internal store name may not be queryable through Interactive Queries.You can retrieve all generated internal topic names via
Topology.describe()
.- Type Parameters:
VR
- the value type of the resultingKTable
- Parameters:
initializer
- anInitializer
that computes an initial intermediate aggregation result. Cannot benull
.aggregator
- anAggregator
that computes a new aggregate result. Cannot benull
.sessionMerger
- aMerger
that combines two aggregation results. Cannot benull
.named
- aNamed
config used to name the processor in the topology. Cannot benull
.- Returns:
- a windowed
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key per session
-
aggregate
<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Merger<? super K, VR> sessionMerger, Materialized<K, VR, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in this stream by the grouped key and defined sessions. Note that sessions are generated on a per-key basis and records with different keys create independent sessions. Records withnull
key or value are ignored. Aggregating is a generalization ofcombining via reduce(...)
as it, for example, allows the result to have a different type than the input values. The result is written into a localSessionStore
(which is basically an ever-updating materialized view) that can be queried using the store name as provided withMaterialized
. Furthermore, updates to the store are sent downstream into aKTable
changelog stream.The specified
Initializer
is applied directly before the first input record per session is processed to provide an initial intermediate aggregation result that is used to process the first record per session. The specifiedAggregator
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 theInitializer
) and the record's value. The specifiedMerger
is used to merge two existing sessions into one, i.e., when the windows overlap, they are merged into a single session and the old sessions are discarded. Thus,aggregate()
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 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 theconfiguration
parameters forcache size
, andcommit interval
.To query the local
SessionStore
it must be obtained viaKafkaStreams#store(...)
:
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // some windowed aggregation on value type double Sting queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance 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)
KafkaStreams.metadataForAllStreamsClients()
to query the value of the key on a parallel running instance of your Kafka Streams application.For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. Therefore, the store name defined by the
Materialized
instance must be a valid Kafka topic name and cannot contain characters other than ASCII alphanumerics, '.', '_' and '-'. The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is user-specified inStreamsConfig
via parameterAPPLICATION_ID_CONFIG
, "storeName" is the provide store name defined inMaterialized
, and "-changelog" is a fixed suffix.You can retrieve all generated internal topic names via
Topology.describe()
.- Type Parameters:
VR
- the value type of the resultingKTable
- Parameters:
initializer
- anInitializer
that computes an initial intermediate aggregation result. Cannot benull
.aggregator
- anAggregator
that computes a new aggregate result. Cannot benull
.sessionMerger
- aMerger
that combines two aggregation results. Cannot benull
.materialized
- aMaterialized
config used to materialize a state store. Cannot benull
.- Returns:
- a windowed
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key per session
-
aggregate
<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Merger<? super K, VR> sessionMerger, Named named, Materialized<K, VR, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in this stream by the grouped key and defined sessions. Note that sessions are generated on a per-key basis and records with different keys create independent sessions. Records withnull
key or value are ignored. Aggregating is a generalization ofcombining via reduce(...)
as it, for example, allows the result to have a different type than the input values. The result is written into a localSessionStore
(which is basically an ever-updating materialized view) that can be queried using the store name as provided withMaterialized
. Furthermore, updates to the store are sent downstream into aKTable
changelog stream.The specified
Initializer
is applied directly before the first input record per session is processed to provide an initial intermediate aggregation result that is used to process the first record per session. The specifiedAggregator
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 theInitializer
) and the record's value. The specifiedMerger
is used to merge two existing sessions into one, i.e., when the windows overlap, they are merged into a single session and the old sessions are discarded. Thus,aggregate()
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 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 theconfiguration
parameters forcache size
, andcommit interval
.To query the local
SessionStore
it must be obtained viaKafkaStreams#store(...)
:
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // some windowed aggregation on value type double Sting queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance 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)
KafkaStreams.metadataForAllStreamsClients()
to query the value of the key on a parallel running instance of your Kafka Streams application.For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. Therefore, the store name defined by the
Materialized
instance must be a valid Kafka topic name and cannot contain characters other than ASCII alphanumerics, '.', '_' and '-'. The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is user-specified inStreamsConfig
via parameterAPPLICATION_ID_CONFIG
, "storeName" is the provide store name defined inMaterialized
, and "-changelog" is a fixed suffix.You can retrieve all generated internal topic names via
Topology.describe()
.- Type Parameters:
VR
- the value type of the resultingKTable
- Parameters:
initializer
- anInitializer
that computes an initial intermediate aggregation result. Cannot benull
.aggregator
- anAggregator
that computes a new aggregate result. Cannot benull
.sessionMerger
- aMerger
that combines two aggregation results. Cannot benull
.named
- aNamed
config used to name the processor in the topology. Cannot benull
.materialized
- aMaterialized
config used to materialize a state store. Cannot benull
.- Returns:
- a windowed
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key per session
-
reduce
Combine the values of records in this stream by the grouped key and defined sessions. Note that sessions are generated on a per-key basis and records with different keys create independent sessions. Records withnull
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)
). The result is written into a localSessionStore
(which is basically an ever-updating materialized view). Furthermore, updates to the store are sent downstream into aKTable
changelog stream. The default key and value serde from the config will be used for serializing the result. If a different serde is required then you should usereduce(Reducer, Materialized)
.The value of the first record per session initialized the session result. The specified
Reducer
is applied for each additional input record per session and computes a new aggregate using the current aggregate (first argument) and the record's value (second argument):
Thus,// At the example of a Reducer<Long> new Reducer<Long>() { public Long apply(Long aggValue, Long currValue) { return aggValue + currValue; } }
reduce()
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 forcache size
, andcommit interval
.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 parameterAPPLICATION_ID_CONFIG
, "internalStoreName" is an internal name and "-changelog" is a fixed suffix.You can retrieve all generated internal topic names via
Topology.describe()
. -
reduce
Combine the values of records in this stream by the grouped key and defined sessions. Note that sessions are generated on a per-key basis and records with different keys create independent sessions. Records withnull
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)
). The result is written into a localSessionStore
(which is basically an ever-updating materialized view). Furthermore, updates to the store are sent downstream into aKTable
changelog stream. The default key and value serde from the config will be used for serializing the result. If a different serde is required then you should usereduce(Reducer, Named, Materialized)
.The value of the first record per session initialized the session result. The specified
Reducer
is applied for each additional input record per session and computes a new aggregate using the current aggregate (first argument) and the record's value (second argument):
Thus,// At the example of a Reducer<Long> new Reducer<Long>() { public Long apply(Long aggValue, Long currValue) { return aggValue + currValue; } }
reduce()
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 forcache size
, andcommit interval
.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 parameterAPPLICATION_ID_CONFIG
, "internalStoreName" is an internal name and "-changelog" is a fixed suffix.You can retrieve all generated internal topic names via
Topology.describe()
.- Parameters:
reducer
- aReducer
that computes a new aggregate result. Cannot benull
.named
- aNamed
config used to name the processor in the topology. Cannot benull
.- Returns:
- a windowed
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key per session
-
reduce
KTable<Windowed<K>,V> reduce(Reducer<V> reducer, Materialized<K, V, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Combine the values of records in this stream by the grouped key and defined sessions. Note that sessions are generated on a per-key basis and records with different keys create independent sessions. Records withnull
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)
). The result is written into a localSessionStore
(which is basically an ever-updating materialized view) that can be queried using the store name as provided withMaterialized
. Furthermore, updates to the store are sent downstream into aKTable
changelog stream.The value of the first record per session initialized the session result. The specified
Reducer
is applied for each additional input record per session and computes a new aggregate using the current aggregate (first argument) and the record's value (second argument):
Thus,// At the example of a Reducer<Long> new Reducer<Long>() { public Long apply(Long aggValue, Long currValue) { return aggValue + currValue; } }
reduce()
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 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 theconfiguration
parameters forcache size
, andcommit interval
.To query the local
SessionStore
it must be obtained viaKafkaStreams#store(...)
:
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // compute sum Sting queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance 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)
KafkaStreams.metadataForAllStreamsClients()
to query the value of the key on a parallel running instance of your Kafka Streams application.For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. Therefore, the store name defined by the Materialized instance must be a valid Kafka topic name and cannot contain characters other than ASCII alphanumerics, '.', '_' and '-'. The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is user-specified in
StreamsConfig
via parameterAPPLICATION_ID_CONFIG
, "storeName" is the provide store name defined inMaterialized
, and "-changelog" is a fixed suffix.You can retrieve all generated internal topic names via
Topology.describe()
.- Parameters:
reducer
- aReducer
that computes a new aggregate result. Cannot benull
.materialized
- aMaterialized
config used to materialize a state store. Cannot benull
.- Returns:
- a windowed
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key per session
-
reduce
KTable<Windowed<K>,V> reduce(Reducer<V> reducer, Named named, Materialized<K, V, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Combine the values of records in this stream by the grouped key and defined sessions. Note that sessions are generated on a per-key basis and records with different keys create independent sessions. Records withnull
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)
). The result is written into a localSessionStore
(which is basically an ever-updating materialized view) that can be queried using the store name as provided withMaterialized
. Furthermore, updates to the store are sent downstream into aKTable
changelog stream.The value of the first record per session initialized the session result. The specified
Reducer
is applied for each additional input record per session and computes a new aggregate using the current aggregate (first argument) and the record's value (second argument):
Thus,// At the example of a Reducer<Long> new Reducer<Long>() { public Long apply(Long aggValue, Long currValue) { return aggValue + currValue; } }
reduce()
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 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 theconfiguration
parameters forcache size
, andcommit interval
.To query the local
SessionStore
it must be obtained viaKafkaStreams.store(StoreQueryParameters)
KafkaStreams#store(...)}:
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // compute sum Sting queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance 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)
KafkaStreams.metadataForAllStreamsClients()
to query the value of the key on a parallel running instance of your Kafka Streams application.For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. Therefore, the store name defined by the Materialized instance must be a valid Kafka topic name and cannot contain characters other than ASCII alphanumerics, '.', '_' and '-'. The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is user-specified in
StreamsConfig
via parameterAPPLICATION_ID_CONFIG
, "storeName" is the provide store name defined inMaterialized
, and "-changelog" is a fixed suffix.You can retrieve all generated internal topic names via
Topology.describe()
.- Parameters:
reducer
- aReducer
that computes a new aggregate result. Cannot benull
.named
- aNamed
config used to name the processor in the topology. Cannot benull
.materialized
- aMaterialized
config used to materialize a state store. Cannot benull
.- Returns:
- a windowed
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key per session
-
emitStrategy
Configure when the aggregated result will be emitted forSessionWindowedKStream
.For example, for
EmitStrategy.onWindowClose()
strategy, the aggregated result for a window will only be emitted when the window closes. ForEmitStrategy.onWindowUpdate()
strategy, the aggregated result for a window will be emitted whenever there is an update to the window. Note that whether the result will be available in downstream also depends on cache policy.- Parameters:
emitStrategy
-EmitStrategy
to configure when the aggregated result for a window will be emitted.- Returns:
- a
SessionWindowedKStream
withEmitStrategy
configured.
-