Interface KGroupedStream<K,V>
- Type Parameters:
K
- Type of keysV
- Type of values
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(...)
.
- See Also:
-
Method Summary
Modifier and TypeMethodDescriptionaggregate
(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator) Aggregate the values of records in this stream by the grouped key.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.aggregate
(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Named named, Materialized<K, VR, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in this stream by the grouped key.<VOut> CogroupedKStream<K,
VOut> cogroup
(Aggregator<? super K, ? super V, VOut> aggregator) Create a newCogroupedKStream
from the this grouped KStream to allow cogrouping otherKGroupedStream
to it.count()
Count the number of records in this stream by the grouped key.count
(Materialized<K, Long, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Count the number of records in this stream by the grouped key.Count the number of records in this stream by the grouped key.count
(Named named, Materialized<K, Long, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Count the number of records in this stream by the grouped key.Combine the values of records in this stream by the grouped key.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.reduce
(Reducer<V> reducer, Named named, Materialized<K, V, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Combine the value of records in this stream by the grouped key.windowedBy
(SessionWindows windows) Create a newSessionWindowedKStream
instance that can be used to perform session windowed aggregations.windowedBy
(SlidingWindows windows) Create a newTimeWindowedKStream
instance that can be used to perform sliding windowed aggregations.<W extends Window>
TimeWindowedKStream<K,V> windowedBy
(Windows<W> windows) Create a newTimeWindowedKStream
instance that can be used to perform windowed aggregations.
-
Method Details
-
count
Count the number of records in this stream by the grouped key. Records withnull
key or value are ignored. The result is written into a localKeyValueStore
(which is basically an ever-updating materialized view). 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 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 (which always will be of type
TimestampedKeyValueStore
) 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 inStreamsConfig
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 viaTopology.describe()
. -
count
Count the number of records in this stream by the grouped key. Records withnull
key or value are ignored. The result is written into a localKeyValueStore
(which is basically an ever-updating materialized view). 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 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 (which always will be of type
TimestampedKeyValueStore
) 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 inStreamsConfig
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 viaTopology.describe()
. -
count
KTable<K,Long> count(Materialized<K, Long, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Count the number of records in this stream by the grouped key. Records withnull
key or value are ignored. The result is written into a localKeyValueStore
(which is basically an ever-updating materialized view) provided by the given store name inmaterialized
. 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 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
.To query the local
ReadOnlyKeyValueStore
it must be obtained viaKafkaStreams#store(...)
.
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // counting words String queryableStoreName = "storeName"; // the store name should be the name of the store as defined by the Materialized instance ReadOnlyKeyValueStore<K, ValueAndTimestamp<Long>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<Long>>timestampedKeyValueStore()); K key = "some-word"; ValueAndTimestamp<Long> countForWord = localStore.get(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 (which always will be of type
TimestampedKeyValueStore
-- regardless of what is specified in the parametermaterialized
) 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 viaTopology.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
KTable
that contains "update" records with unmodified keys andLong
values that represent the latest (rolling) count (i.e., number of records) for each key
-
count
KTable<K,Long> count(Named named, Materialized<K, Long, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Count the number of records in this stream by the grouped key. Records withnull
key or value are ignored. The result is written into a localKeyValueStore
(which is basically an ever-updating materialized view) provided by the given store name inmaterialized
. 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 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
.To query the local
ReadOnlyKeyValueStore
it must be obtained viaKafkaStreams#store(...)
.
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // counting words String queryableStoreName = "storeName"; // the store name should be the name of the store as defined by the Materialized instance ReadOnlyKeyValueStore<K, ValueAndTimestamp<Long>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<Long>>timestampedKeyValueStore()); K key = "some-word"; ValueAndTimestamp<Long> countForWord = localStore.get(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 (which always will be of type
TimestampedKeyValueStore
-- regardless of what is specified in the parametermaterialized
) 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 viaTopology.describe()
.- Parameters:
named
- aNamed
config used to name the processor in the topologymaterialized
- 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
KTable
that contains "update" records with unmodified keys andLong
values that represent the latest (rolling) count (i.e., number of records) for each key
-
reduce
Combine the values of records in this stream by the grouped key. 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)
).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 theReducer
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 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 (which always will be of type
TimestampedKeyValueStore
) 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 inStreamsConfig
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 viaTopology.describe()
.- Parameters:
reducer
- aReducer
that computes a new aggregate result. Cannot benull
.- Returns:
- a
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key. If the reduce function returnsnull
, it is then interpreted as deletion for the key, and future messages of the same key coming from upstream operators will be handled as newly initialized value.
-
reduce
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. 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, Materialized)
). The result is written into a localKeyValueStore
(which is basically an ever-updating materialized view) provided by the given store name inmaterialized
. Furthermore, updates to the store are sent downstream into aKTable
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 forcache size
, andcommit interval
.To query the local
ReadOnlyKeyValueStore
it must be obtained viaKafkaStreams#store(...)
.
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // compute sum String queryableStoreName = "storeName" // the store name should be the name of the store as defined by the Materialized instance ReadOnlyKeyValueStore<K, ValueAndTimestamp<V>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<V>>timestampedKeyValueStore()); K key = "some-key"; ValueAndTimestamp<V> reduceForKey = localStore.get(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 (which always will be of type
TimestampedKeyValueStore
-- regardless of what is specified in the parametermaterialized
) 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 inStreamsConfig
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 viaTopology.describe()
.- Parameters:
reducer
- aReducer
that computes a new aggregate result. Cannot benull
.materialized
- an instance ofMaterialized
used to materialize a state store. Cannot benull
.- Returns:
- a
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key
-
reduce
KTable<K,V> reduce(Reducer<V> reducer, Named named, Materialized<K, V, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Combine the value of records in this stream by the grouped key. 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, Materialized)
). The result is written into a localKeyValueStore
(which is basically an ever-updating materialized view) provided by the given store name inmaterialized
. Furthermore, updates to the store are sent downstream into aKTable
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 forcache size
, andcommit interval
.To query the local
ReadOnlyKeyValueStore
it must be obtained viaKafkaStreams#store(...)
.
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // compute sum String queryableStoreName = "storeName" // the store name should be the name of the store as defined by the Materialized instance ReadOnlyKeyValueStore<K, ValueAndTimestamp<V>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<V>>timestampedKeyValueStore()); K key = "some-key"; ValueAndTimestamp<V> reduceForKey = localStore.get(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 (which always will be of type
TimestampedKeyValueStore
-- regardless of what is specified in the parametermaterialized
) 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 inStreamsConfig
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 viaTopology.describe()
.- Parameters:
reducer
- aReducer
that computes a new aggregate result. Cannot benull
.named
- aNamed
config used to name the processor in the topology.materialized
- an instance ofMaterialized
used to materialize a state store. Cannot benull
.- Returns:
- a
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key. If the reduce function returnsnull
, it is then interpreted as deletion for the key, and future messages of the same key coming from upstream operators will be handled as newly initialized value.
-
aggregate
<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. 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 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 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. 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 forcache size
, andcommit interval
.For failure and recovery the store (which always will be of type
TimestampedKeyValueStore
) 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 inStreamsConfig
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 viaTopology.describe()
.- Type Parameters:
VR
- the value type of the resultingKTable
- Parameters:
initializer
- anInitializer
that computes an initial intermediate aggregation resultaggregator
- anAggregator
that computes a new aggregate result- Returns:
- a
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key. If the aggregate function returnsnull
, it is then interpreted as deletion for the key, and future messages of the same key coming from upstream operators will be handled as newly initialized value.
-
aggregate
<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. 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 localKeyValueStore
(which is basically an ever-updating materialized view) that can be queried by the given store name inmaterialized
. Furthermore, updates to the store are sent downstream into aKTable
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 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. Thus,aggregate(Initializer, Aggregator, Materialized)
can be used to compute aggregate functions like count (c.f.count()
).Not all updates might get sent downstream, as an internal cache 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 forcache size
, andcommit interval
.To query the local
ReadOnlyKeyValueStore
it must be obtained viaKafkaStreams#store(...)
:
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // some aggregation on value type double String queryableStoreName = "storeName" // the store name should be the name of the store as defined by the Materialized instance ReadOnlyKeyValueStore<K, ValueAndTimestamp<VR>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<VR>>timestampedKeyValueStore()); K key = "some-key"; ValueAndTimestamp<VR> aggForKey = localStore.get(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 (which always will be of type
TimestampedKeyValueStore
-- regardless of what is specified in the parametermaterialized
) 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 viaTopology.describe()
.- Type Parameters:
VR
- the value type of the resultingKTable
- Parameters:
initializer
- anInitializer
that computes an initial intermediate aggregation resultaggregator
- anAggregator
that computes a new aggregate resultmaterialized
- an instance ofMaterialized
used to materialize a state store. Cannot benull
.- Returns:
- a
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key
-
aggregate
<VR> KTable<K,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Named named, Materialized<K, VR, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in this stream by the grouped key. 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 localKeyValueStore
(which is basically an ever-updating materialized view) that can be queried by the given store name inmaterialized
. Furthermore, updates to the store are sent downstream into aKTable
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 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. Thus,aggregate(Initializer, Aggregator, Materialized)
can be used to compute aggregate functions like count (c.f.count()
).Not all updates might get sent downstream, as an internal cache 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 forcache size
, andcommit interval
.To query the local
ReadOnlyKeyValueStore
it must be obtained viaKafkaStreams#store(...)
:
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // some aggregation on value type double String queryableStoreName = "storeName" // the store name should be the name of the store as defined by the Materialized instance ReadOnlyKeyValueStore<K, ValueAndTimestamp<VR>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, ValueAndTimestamp<VR>>timestampedKeyValueStore()); K key = "some-key"; ValueAndTimestamp<VR> aggForKey = localStore.get(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 (which always will be of type
TimestampedKeyValueStore
-- regardless of what is specified in the parametermaterialized
) 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 viaTopology.describe()
.- Type Parameters:
VR
- the value type of the resultingKTable
- Parameters:
initializer
- anInitializer
that computes an initial intermediate aggregation resultaggregator
- anAggregator
that computes a new aggregate resultnamed
- aNamed
config used to name the processor in the topologymaterialized
- an instance ofMaterialized
used to materialize a state store. Cannot benull
.- Returns:
- a
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key. If the aggregate function returnsnull
, it is then interpreted as deletion for the key, and future messages of the same key coming from upstream operators will be handled as newly initialized value.
-
windowedBy
Create a newTimeWindowedKStream
instance that can be used to perform windowed aggregations.- Type Parameters:
W
- the window type- Parameters:
windows
- the specification of the aggregationWindows
- Returns:
- an instance of
TimeWindowedKStream
-
windowedBy
Create a newTimeWindowedKStream
instance that can be used to perform sliding windowed aggregations.- Parameters:
windows
- the specification of the aggregationSlidingWindows
- Returns:
- an instance of
TimeWindowedKStream
-
windowedBy
Create a newSessionWindowedKStream
instance that can be used to perform session windowed aggregations.- Parameters:
windows
- the specification of the aggregationSessionWindows
- Returns:
- an instance of
TimeWindowedKStream
-
cogroup
Create a newCogroupedKStream
from the this grouped KStream to allow cogrouping otherKGroupedStream
to it.CogroupedKStream
is an abstraction of multiple grouped record streams ofKeyValue
pairs. It is an intermediate representation after a grouping ofKStream
s, before the aggregations are applied to the new partitions resulting in aKTable
.The specified
Aggregator
is applied in the actualaggregation
step for each input record and computes a new aggregate using the current aggregate (or for the very first record per key using the initial intermediate aggregation result provided via theInitializer
that is passed intoCogroupedKStream.aggregate(Initializer)
) and the record's value.- Type Parameters:
VOut
- the type of the output values- Parameters:
aggregator
- anAggregator
that computes a new aggregate result- Returns:
- a
CogroupedKStream
-