Interface CogroupedKStream<K,VOut>
- Type Parameters:
K
- Type of keysVOut
- Type of values after agg
CogroupedKStream
is an abstraction of multiple grouped record streams of KeyValue
pairs.
It is an intermediate representation after a grouping of KStream
s, before the
aggregations are applied to the new partitions resulting in a KTable
.
A CogroupedKStream
must be obtained from a KGroupedStream
via
cogroup(...)
.
-
Method Summary
Modifier and TypeMethodDescriptionaggregate
(Initializer<VOut> initializer) Aggregate the values of records in these streams by the grouped key.aggregate
(Initializer<VOut> initializer, Materialized<K, VOut, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in these streams by the grouped key.aggregate
(Initializer<VOut> initializer, Named named) Aggregate the values of records in these streams by the grouped key.aggregate
(Initializer<VOut> initializer, Named named, Materialized<K, VOut, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in these streams by the grouped key.<VIn> CogroupedKStream<K,
VOut> cogroup
(KGroupedStream<K, VIn> groupedStream, Aggregator<? super K, ? super VIn, VOut> aggregator) Add an alreadygrouped KStream
to thisCogroupedKStream
.windowedBy
(SessionWindows windows) Create a newSessionWindowedCogroupedKStream
instance that can be used to perform session windowed aggregations.windowedBy
(SlidingWindows windows) Create a newTimeWindowedCogroupedKStream
instance that can be used to perform sliding windowed aggregations.<W extends Window>
TimeWindowedCogroupedKStream<K,VOut> windowedBy
(Windows<W> windows) Create a newTimeWindowedCogroupedKStream
instance that can be used to perform windowed aggregations.
-
Method Details
-
cogroup
<VIn> CogroupedKStream<K,VOut> cogroup(KGroupedStream<K, VIn> groupedStream, Aggregator<? super K, ? super VIn, VOut> aggregator) Add an alreadygrouped KStream
to thisCogroupedKStream
.The added
grouped KStream
must have the same number of partitions as all existing streams of thisCogroupedKStream
. If this is not the case, you would need to callKStream.repartition(Repartitioned)
beforegrouping
theKStream
and specify the "correct" number of partitions viaRepartitioned
parameter.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 intoaggregate(Initializer)
) and the record's value.- Type Parameters:
VIn
- Type of input values- Parameters:
groupedStream
- a group streamaggregator
- anAggregator
that computes a new aggregate result- Returns:
- a
CogroupedKStream
-
aggregate
Aggregate the values of records in these streams 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) that can be queried by the given store name inmaterialized
. Furthermore, updates to the store are sent downstream into aKTable
changelog stream.To compute the aggregation the corresponding
Aggregator
as specified incogroup(...)
is used per input stream. The specifiedInitializer
is applied once per key, directly before the first input record per key is processed to provide an initial intermediate aggregation result that is used to process the first record.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<VOut>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<VOut>> timestampedKeyValueStore()); K key = "some-key"; ValueAndTimestamp<VOut> 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
) 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 a generated value, and "-changelog" is a fixed suffix.You can retrieve all generated internal topic names via
Topology.describe()
.- Parameters:
initializer
- anInitializer
that computes an initial intermediate aggregation result. Cannot benull
.- Returns:
- a
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key
-
aggregate
Aggregate the values of records in these streams 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) that can be queried by the given store name inmaterialized
. Furthermore, updates to the store are sent downstream into aKTable
changelog stream.To compute the aggregation the corresponding
Aggregator
as specified incogroup(...)
is used per input stream. The specifiedInitializer
is applied once per key, directly before the first input record per key is processed to provide an initial intermediate aggregation result that is used to process the first record. The specifiedNamed
is applied once to the processor combining the grouped streams.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<VOut>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<VOut>> timestampedKeyValueStore()); K key = "some-key"; ValueAndTimestamp<VOut> 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
) 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()
.- Parameters:
initializer
- anInitializer
that computes an initial intermediate aggregation result. Cannot benull
.named
- name the processor. Cannot benull
.- Returns:
- a
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key
-
aggregate
KTable<K,VOut> aggregate(Initializer<VOut> initializer, Materialized<K, VOut, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in these streams 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) that can be queried by the given store name inmaterialized
. Furthermore, updates to the store are sent downstream into aKTable
changelog stream.To compute the aggregation the corresponding
Aggregator
as specified incogroup(...)
is used per input stream. The specifiedInitializer
is applied once per key, directly before the first input record per key is processed to provide an initial intermediate aggregation result that is used to process the first record.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<VOut>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<VOut>> timestampedKeyValueStore()); K key = "some-key"; ValueAndTimestamp<VOut> 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 via
Topology.describe()
.- Parameters:
initializer
- anInitializer
that computes an initial intermediate aggregation 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
-
aggregate
KTable<K,VOut> aggregate(Initializer<VOut> initializer, Named named, Materialized<K, VOut, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in these streams 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) that can be queried by the given store name inmaterialized
. Furthermore, updates to the store are sent downstream into aKTable
changelog stream.To compute the aggregation the corresponding
Aggregator
as specified incogroup(...)
is used per input stream. The specifiedInitializer
is applied once per key, directly before the first input record per key is processed to provide an initial intermediate aggregation result that is used to process the first record. The specifiedNamed
is used to name the processor combining the grouped streams.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<VOut>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<VOut>> timestampedKeyValueStore()); K key = "some-key"; ValueAndTimestamp<VOut> 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 via
Topology.describe()
.- Parameters:
initializer
- anInitializer
that computes an initial intermediate aggregation result. Cannot benull
.materialized
- an instance ofMaterialized
used to materialize a state store. Cannot benull
.named
- name the processors. Cannot benull
.- Returns:
- a
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key
-
windowedBy
Create a newTimeWindowedCogroupedKStream
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
TimeWindowedCogroupedKStream
-
windowedBy
Create a newTimeWindowedCogroupedKStream
instance that can be used to perform sliding windowed aggregations.- Parameters:
windows
- the specification of the aggregationSlidingWindows
- Returns:
- an instance of
TimeWindowedCogroupedKStream
-
windowedBy
Create a newSessionWindowedCogroupedKStream
instance that can be used to perform session windowed aggregations.- Parameters:
windows
- the specification of the aggregationSessionWindows
- Returns:
- an instance of
SessionWindowedCogroupedKStream
-