K
- Type of keysV
- Type of valuespublic interface SessionWindowedKStream<K,V>
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.
Please see SessionWindows
for more details.
New events are added to SessionWindows
until their grace period ends (see SessionWindows.grace(Duration)
).
Furthermore, updates are sent downstream into a windowed KTable
changelog stream, where
"windowed" implies that the KTable
key is a combined key of the original record key and a window ID.
A SessionWindowedKStream
must be obtained from a KGroupedStream
via KGroupedStream.windowedBy(SessionWindows)
.
KStream
,
KGroupedStream
,
SessionWindows
Modifier and Type | Method and Description |
---|---|
<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
SessionWindows . |
<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
SessionWindows . |
KTable<Windowed<K>,java.lang.Long> |
count()
Count the number of records in this stream by the grouped key into
SessionWindows . |
KTable<Windowed<K>,java.lang.Long> |
count(Materialized<K,java.lang.Long,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Count the number of records in this stream by the grouped key into
SessionWindows . |
KTable<Windowed<K>,V> |
reduce(Reducer<V> reducer)
Combine values of this stream by the grouped key into
SessionWindows . |
KTable<Windowed<K>,V> |
reduce(Reducer<V> reducer,
Materialized<K,V,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materializedAs)
Combine values of this stream by the grouped key into
SessionWindows . |
KTable<Windowed<K>,java.lang.Long> count()
SessionWindows
.
Records with null
key or value are ignored.
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same window and key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration
parameters for
cache size
, and
commit intervall
.
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<Windowed<K>,java.lang.Long> count(Materialized<K,java.lang.Long,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
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 name provided with Materialized
.
Not all updates might get sent downstream, as an internal cache will be used to deduplicate consecutive updates to
the same window and key if caching is enabled on the Materialized
instance.
When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration
parameters for
cache size
, and
commit intervall
To query the local windowed KeyValueStore
it must be obtained via
KafkaStreams#store(...)
.
KafkaStreams streams = ... // 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)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata()
to
query the value of the key on a parallel running instance of your Kafka Streams application.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
Therefore, the store name defined by the Materialized instance must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "storeName" is the
provide store name defined in Materialized
, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via Topology.describe()
.
materialized
- an instance of Materialized
used to materialize a state store. Cannot be null
.
Note: the valueSerde will be automatically set to Serdes.Long()
if there is no valueSerde providedKTable
that contains "update" records with unmodified keys and Long
values
that represent the latest (rolling) count (i.e., number of records) for each key within a window<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Merger<? super K,VR> sessionMerger)
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 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.
The specified Merger
is used to merge 2 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(Initializer, Aggregator, Merger)
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, 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 for
cache size
, and
commit intervall
.
VR
- the value type of the resulting KTable
initializer
- the instance of Initializer
aggregator
- the instance of Aggregator
sessionMerger
- the instance of Merger
KTable
that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window<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)
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 name provided with Materialized
.
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 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.
* The specified Merger
is used to merge 2 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(Initializer, Aggregator, Merger)
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 the configuration
parameters for
cache size
, and
commit intervall
KafkaStreams 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)
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
Therefore, the store name defined by the Materialized instance must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "storeName" is the
provide store name defined in Materialized
, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via Topology.describe()
.
VR
- the value type of the resulting KTable
initializer
- the instance of Initializer
aggregator
- the instance of Aggregator
sessionMerger
- the instance of Merger
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 within a windowKTable<Windowed<K>,V> reduce(Reducer<V> 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)
).
The result is written into a local SessionStore
(which is basically an ever-updating
materialized view).
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)
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
.
KTable<Windowed<K>,V> reduce(Reducer<V> reducer, Materialized<K,V,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materializedAs)
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)
).
The result is written into a local SessionStore
(which is basically an ever-updating materialized view)
provided by the given Materialized
instance.
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 will be used to deduplicate consecutive updates to
the same window and key if caching is enabled on the Materialized
instance.
When caching is enabled the rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration
parameters for
cache size
, and
commit intervall
To query the local windowed KeyValueStore
it must be obtained via
KafkaStreams#store(...)
.
KafkaStreams streams = ... // 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)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata()
to
query the value of the key on a parallel running instance of your Kafka Streams application.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
Therefore, the store name defined by the Materialized instance must be a valid Kafka topic name and cannot contain characters other than ASCII
alphanumerics, '.', '_' and '-'.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "storeName" is the
provide store name defined in Materialized
, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via Topology.describe()
.
reducer
- a Reducer
that computes a new aggregate result. Cannot be null
.materializedAs
- 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 within a window