Interface CogroupedKStream<K,VAgg>
- Type Parameters:
K- Type of keysVAgg- 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 KStreams, 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<VAgg> initializer) Aggregate the values of records in these streams by the grouped key.aggregate(Initializer<VAgg> initializer, Materialized<K, VAgg, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in these streams by the grouped key.aggregate(Initializer<VAgg> initializer, Named named) Aggregate the values of records in these streams by the grouped key.aggregate(Initializer<VAgg> initializer, Named named, Materialized<K, VAgg, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in these streams by the grouped key.<V> CogroupedKStream<K,VAgg> cogroup(KGroupedStream<K, V> groupedStream, Aggregator<? super K, ? super V, VAgg> aggregator) Add an alreadygrouped KStreamto thisCogroupedKStream.windowedBy(SessionWindows windows) Create a newSessionWindowedCogroupedKStreaminstance that can be used to perform session windowed aggregations.windowedBy(SlidingWindows windows) Create a newTimeWindowedCogroupedKStreaminstance that can be used to perform sliding windowed aggregations.<W extends Window>
TimeWindowedCogroupedKStream<K,VAgg> windowedBy(Windows<W> windows) Create a newTimeWindowedCogroupedKStreaminstance that can be used to perform windowed aggregations.
-
Method Details
-
cogroup
<V> CogroupedKStream<K,VAgg> cogroup(KGroupedStream<K, V> groupedStream, Aggregator<? super K, ? super V, VAgg> aggregator) Add an alreadygrouped KStreamto thisCogroupedKStream.The added
grouped KStreammust 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)beforegroupingtheKStreamand specify the "correct" number of partitions viaRepartitionedparameter.The specified
Aggregatoris applied in the actualaggregationstep 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 theInitializerthat is passed intoaggregate(Initializer)) and the record's value.- Type Parameters:
V- Type of input values- Parameters:
groupedStream- a group streamaggregator- anAggregatorthat computes a new aggregate result- Returns:
- a
CogroupedKStream
-
aggregate
Aggregate the values of records in these streams by the grouped key. Records withnullkey 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 aKTablechangelog stream.To compute the aggregation the corresponding
Aggregatoras specified incogroup(...)is used per input stream. The specifiedInitializeris 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
configurationparameters forcache size, andcommit interval.To query the local
ReadOnlyKeyValueStoreit 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 StoreQueryParameters<ReadOnlyKeyValueStore<K, ValueAndTimestamp<VOut>>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedKeyValueStore()); ReadOnlyKeyValueStore<K, ValueAndTimestamp<VOut>> localStore = streams.store(storeQueryParams); 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 inStreamsConfigvia 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- anInitializerthat computes an initial intermediate aggregation result. Cannot benull.- Returns:
- a
KTablethat 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 withnullkey 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 aKTablechangelog stream.To compute the aggregation the corresponding
Aggregatoras specified incogroup(...)is used per input stream. The specifiedInitializeris 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 specifiedNamedis 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
configurationparameters forcache size, andcommit interval.To query the local
ReadOnlyKeyValueStoreit 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 StoreQueryParameters<ReadOnlyKeyValueStore<K, ValueAndTimestamp<VOut>>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedKeyValueStore()); ReadOnlyKeyValueStore<K, ValueAndTimestamp<VOut>> localStore = streams.store(storeQueryParams); 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 inStreamsConfigvia 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- anInitializerthat computes an initial intermediate aggregation result. Cannot benull.named- name the processor. Cannot benull.- Returns:
- a
KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key
-
aggregate
KTable<K,VAgg> aggregate(Initializer<VAgg> initializer, Materialized<K, VAgg, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in these streams by the grouped key. Records withnullkey 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 aKTablechangelog stream.To compute the aggregation the corresponding
Aggregatoras specified incogroup(...)is used per input stream. The specifiedInitializeris 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
configurationparameters forcache size, andcommit interval.To query the local
ReadOnlyKeyValueStoreit 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 StoreQueryParameters<ReadOnlyKeyValueStore<K, ValueAndTimestamp<VOut>>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedKeyValueStore()); ReadOnlyKeyValueStore<K, ValueAndTimestamp<VOut>> localStore = streams.store(storeQueryParams); 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 inStreamsConfigvia 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- anInitializerthat computes an initial intermediate aggregation result. Cannot benull.materialized- an instance ofMaterializedused to materialize a state store. Cannot benull.- Returns:
- a
KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key
-
aggregate
KTable<K,VAgg> aggregate(Initializer<VAgg> initializer, Named named, Materialized<K, VAgg, KeyValueStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in these streams by the grouped key. Records withnullkey 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 aKTablechangelog stream.To compute the aggregation the corresponding
Aggregatoras specified incogroup(...)is used per input stream. The specifiedInitializeris 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 specifiedNamedis 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
configurationparameters forcache size, andcommit interval.To query the local
ReadOnlyKeyValueStoreit 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 StoreQueryParameters<ReadOnlyKeyValueStore<K, ValueAndTimestamp<VOut>>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedKeyValueStore()); ReadOnlyKeyValueStore<K, ValueAndTimestamp<VOut>> localStore = streams.store(storeQueryParams); 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 inStreamsConfigvia 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- anInitializerthat computes an initial intermediate aggregation result. Cannot benull.materialized- an instance ofMaterializedused to materialize a state store. Cannot benull.named- name the processors. Cannot benull.- Returns:
- a
KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key
-
windowedBy
Create a newTimeWindowedCogroupedKStreaminstance 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 newTimeWindowedCogroupedKStreaminstance that can be used to perform sliding windowed aggregations.- Parameters:
windows- the specification of the aggregationSlidingWindows- Returns:
- an instance of
TimeWindowedCogroupedKStream
-
windowedBy
Create a newSessionWindowedCogroupedKStreaminstance that can be used to perform session windowed aggregations.- Parameters:
windows- the specification of the aggregationSessionWindows- Returns:
- an instance of
SessionWindowedCogroupedKStream
-