K - Type of keysVAgg - Type of values after aggpublic interface CogroupedKStream<K,VAgg>
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(...).
| Modifier and Type | Method and Description | 
|---|---|
| KTable<K,VAgg> | aggregate(Initializer<VAgg> initializer)Aggregate the values of records in these streams by the grouped key. | 
| 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. | 
| KTable<K,VAgg> | aggregate(Initializer<VAgg> initializer,
         Named named)Aggregate the values of records in these streams by the grouped key. | 
| 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. | 
| <V> CogroupedKStream<K,VAgg> | cogroup(KGroupedStream<K,V> groupedStream,
       Aggregator<? super K,? super V,VAgg> aggregator)Add an already  grouped KStreamto thisCogroupedKStream. | 
| SessionWindowedCogroupedKStream<K,VAgg> | windowedBy(SessionWindows windows)Create a new  SessionWindowedCogroupedKStreaminstance that can be used to perform session
 windowed aggregations. | 
| TimeWindowedCogroupedKStream<K,VAgg> | windowedBy(SlidingWindows windows)Create a new  TimeWindowedCogroupedKStreaminstance that can be used to perform sliding
 windowed aggregations. | 
| <W extends Window> | windowedBy(Windows<W> windows)Create a new  TimeWindowedCogroupedKStreaminstance that can be used to perform windowed
 aggregations. | 
<V> CogroupedKStream<K,VAgg> cogroup(KGroupedStream<K,V> groupedStream, Aggregator<? super K,? super V,VAgg> aggregator)
grouped KStream to this CogroupedKStream.
 
 The added grouped KStream must have the same number of partitions as all existing
 streams of this CogroupedKStream.
 If this is not the case, you would need to call KStream.repartition(Repartitioned) before
 grouping the KStream and specify the "correct" number of
 partitions via Repartitioned parameter.
 
 The specified Aggregator is applied in the actual aggregation 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 the Initializer that is passed into
 aggregate(Initializer)) and the record's value.
V - Type of input valuesgroupedStream - a group streamaggregator - an Aggregator that computes a new aggregate resultCogroupedKStreamKTable<K,VAgg> aggregate(Initializer<VAgg> initializer)
null key or value are ignored.
 The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
 that can be queried by the given store name in materialized.
 Furthermore, updates to the store are sent downstream into a KTable changelog stream.
 
 To compute the aggregation the corresponding Aggregator as specified in
 cogroup(...) is used per input stream.
 The specified Initializer 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 for
 cache size, and
 commit interval.
 
 To query the local ReadOnlyKeyValueStore it must be obtained via
 KafkaStreams#store(...):
 
 KafkaStreams 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 in StreamsConfig via parameter
 APPLICATION_ID_CONFIG, "storeName" is a generated value, and
 "-changelog" is a fixed suffix.
 
 You can retrieve all generated internal topic names via Topology.describe().
initializer - an Initializer that computes an initial intermediate aggregation
        result. Cannot be null.KTable that contains "update" records with unmodified keys, and values that
 represent the latest (rolling) aggregate for each keyKTable<K,VAgg> aggregate(Initializer<VAgg> initializer, Named named)
null key or value are ignored.
 The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
 that can be queried by the given store name in materialized.
 Furthermore, updates to the store are sent downstream into a KTable changelog stream.
 
 To compute the aggregation the corresponding Aggregator as specified in
 cogroup(...) is used per input stream.
 The specified Initializer 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 specified Named 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 for
 cache size, and
 commit interval.
 
 To query the local ReadOnlyKeyValueStore it must be obtained via
 KafkaStreams#store(...):
 
 KafkaStreams 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 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().
initializer - an Initializer that computes an initial intermediate aggregation result. Cannot be null.named - name the processor. Cannot be null.KTable that contains "update" records with unmodified keys, and values that
 represent the latest (rolling) aggregate for each keyKTable<K,VAgg> aggregate(Initializer<VAgg> initializer, Materialized<K,VAgg,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
null key or value are ignored.
 The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
 that can be queried by the given store name in materialized.
 Furthermore, updates to the store are sent downstream into a KTable changelog stream.
 
 To compute the aggregation the corresponding Aggregator as specified in
 cogroup(...) is used per input stream.
 The specified Initializer 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 for
 cache size, and
 commit interval.
 
 To query the local ReadOnlyKeyValueStore it must be obtained via
 KafkaStreams#store(...):
 
 KafkaStreams 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 parameter materialized) 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().
initializer - an Initializer that computes an initial intermediate aggregation result. Cannot be null.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 keyKTable<K,VAgg> aggregate(Initializer<VAgg> initializer, Named named, Materialized<K,VAgg,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
null key or value are ignored.
 The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
 that can be queried by the given store name in materialized.
 Furthermore, updates to the store are sent downstream into a KTable changelog stream.
 
 To compute the aggregation the corresponding Aggregator as specified in
 cogroup(...) is used per input stream.
 The specified Initializer 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 specified Named 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 for
 cache size, and
 commit interval.
 
 To query the local ReadOnlyKeyValueStore it must be obtained via
 KafkaStreams#store(...):
 
 KafkaStreams 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 parameter materialized) 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().
initializer - an Initializer that computes an initial intermediate aggregation result. Cannot be null.materialized - an instance of Materialized used to materialize a state store. Cannot be null.named - name the processors. Cannot be null.KTable that contains "update" records with unmodified keys, and values that
 represent the latest (rolling) aggregate for each key<W extends Window> TimeWindowedCogroupedKStream<K,VAgg> windowedBy(Windows<W> windows)
TimeWindowedCogroupedKStream instance that can be used to perform windowed
 aggregations.W - the window typewindows - the specification of the aggregation WindowsTimeWindowedCogroupedKStreamTimeWindowedCogroupedKStream<K,VAgg> windowedBy(SlidingWindows windows)
TimeWindowedCogroupedKStream instance that can be used to perform sliding
 windowed aggregations.windows - the specification of the aggregation SlidingWindowsTimeWindowedCogroupedKStreamSessionWindowedCogroupedKStream<K,VAgg> windowedBy(SessionWindows windows)
SessionWindowedCogroupedKStream instance that can be used to perform session
 windowed aggregations.windows - the specification of the aggregation SessionWindowsSessionWindowedCogroupedKStream