Interface CogroupedKStream<K,VAgg> 
- Type Parameters:
- K- Type of keys
- VAgg- 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 SummaryModifier 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 stream
- aggregator- an- Aggregatorthat computes a new aggregate result
- Returns:
- a CogroupedKStream
 
- 
aggregateAggregate 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- an- Initializerthat computes an initial intermediate aggregation result. Cannot be- null.
- Returns:
- a KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key
 
- 
aggregateAggregate 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- an- Initializerthat computes an initial intermediate aggregation result. Cannot be- null.
- named- name the processor. Cannot be- null.
- Returns:
- a KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key
 
- 
aggregateKTable<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- an- Initializerthat computes an initial intermediate aggregation result. Cannot be- null.
- materialized- an instance of- Materializedused to materialize a state store. Cannot be- null.
- Returns:
- a KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key
 
- 
aggregateKTable<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- an- Initializerthat computes an initial intermediate aggregation result. Cannot be- null.
- named- name the processors. Cannot be- null.
- materialized- an instance of- Materializedused to materialize a state store. Cannot be- null.
- Returns:
- a KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key
 
- 
windowedByCreate a newTimeWindowedCogroupedKStreaminstance that can be used to perform windowed aggregations.- Type Parameters:
- W- the window type
- Parameters:
- windows- the specification of the aggregation- Windows
- Returns:
- an instance of TimeWindowedCogroupedKStream
 
- 
windowedByCreate a newTimeWindowedCogroupedKStreaminstance that can be used to perform sliding windowed aggregations.- Parameters:
- windows- the specification of the aggregation- SlidingWindows
- Returns:
- an instance of TimeWindowedCogroupedKStream
 
- 
windowedByCreate a newSessionWindowedCogroupedKStreaminstance that can be used to perform session windowed aggregations.- Parameters:
- windows- the specification of the aggregation- SessionWindows
- Returns:
- an instance of SessionWindowedCogroupedKStream
 
 
-