Interface KGroupedStream<K,V>

Type Parameters:
K - Type of keys
V - Type of values

public interface KGroupedStream<K,V>
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 Details

    • count

      KTable<K,Long> count()
      Count the number of records in this stream by the grouped key. Records with null key or value are ignored. The result is written into a local KeyValueStore (which is basically an ever-updating materialized view). Furthermore, updates to the store are sent downstream into a KTable 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 for cache size, and commit 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 in StreamsConfig via parameter APPLICATION_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 via Topology.describe().

      Returns:
      a KTable that contains "update" records with unmodified keys and Long values that represent the latest (rolling) count (i.e., number of records) for each key
    • count

      KTable<K,Long> count(Named named)
      Count the number of records in this stream by the grouped key. Records with null key or value are ignored. The result is written into a local KeyValueStore (which is basically an ever-updating materialized view). Furthermore, updates to the store are sent downstream into a KTable 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 for cache size, and commit 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 in StreamsConfig via parameter APPLICATION_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 via Topology.describe().

      Parameters:
      named - a Named config used to name the processor in the topology
      Returns:
      a KTable that contains "update" records with unmodified keys and Long values that represent the latest (rolling) count (i.e., number of records) for each key
    • 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 with null key or value are ignored. The result is written into a local KeyValueStore (which is basically an ever-updating materialized view) provided by the given store name in materialized. Furthermore, updates to the store are sent downstream into a KTable 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 for cache size, and commit interval.

      To query the local ReadOnlyKeyValueStore it must be obtained via KafkaStreams#store(...).

      
       KafkaStreams 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)
       
      For non-local keys, a custom RPC mechanism must be implemented using 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().

      Parameters:
      materialized - an instance of Materialized used to materialize a state store. Cannot be null. Note: the valueSerde will be automatically set to Serdes#Long() if there is no valueSerde provided
      Returns:
      a KTable that contains "update" records with unmodified keys and Long 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 with null key or value are ignored. The result is written into a local KeyValueStore (which is basically an ever-updating materialized view) provided by the given store name in materialized. Furthermore, updates to the store are sent downstream into a KTable 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 for cache size, and commit interval.

      To query the local ReadOnlyKeyValueStore it must be obtained via KafkaStreams#store(...).

      
       KafkaStreams 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)
       
      For non-local keys, a custom RPC mechanism must be implemented using 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().

      Parameters:
      named - a Named config used to name the processor in the topology
      materialized - an instance of Materialized used to materialize a state store. Cannot be null. Note: the valueSerde will be automatically set to Serdes#Long() if there is no valueSerde provided
      Returns:
      a KTable that contains "update" records with unmodified keys and Long values that represent the latest (rolling) count (i.e., number of records) for each key
    • reduce

      KTable<K,V> reduce(Reducer<V> reducer)
      Combine the values of records in this stream by the grouped key. Records with null 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 the Reducer 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 for cache size, and commit 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 in StreamsConfig via parameter APPLICATION_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 via Topology.describe().

      Parameters:
      reducer - a Reducer that computes a new aggregate result. Cannot be null.
      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 returns null, 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 with null 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 local KeyValueStore (which is basically an ever-updating materialized view) provided by the given store name in materialized. Furthermore, updates to the store are sent downstream into a KTable 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 for cache size, and commit interval.

      To query the local ReadOnlyKeyValueStore it must be obtained via KafkaStreams#store(...).

      
       KafkaStreams 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)
       
      For non-local keys, a custom RPC mechanism must be implemented using 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. The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is user-specified in StreamsConfig via parameter APPLICATION_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 via Topology.describe().

      Parameters:
      reducer - a Reducer that computes a new aggregate result. Cannot be null.
      materialized - an instance of Materialized used to materialize a state store. Cannot be null.
      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 with null 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 local KeyValueStore (which is basically an ever-updating materialized view) provided by the given store name in materialized. Furthermore, updates to the store are sent downstream into a KTable 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 for cache size, and commit interval.

      To query the local ReadOnlyKeyValueStore it must be obtained via KafkaStreams#store(...).

      
       KafkaStreams 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)
       
      For non-local keys, a custom RPC mechanism must be implemented using 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. The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is user-specified in StreamsConfig via parameter APPLICATION_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 via Topology.describe().

      Parameters:
      reducer - a Reducer that computes a new aggregate result. Cannot be null.
      named - a Named config used to name the processor in the topology.
      materialized - an instance of Materialized used to materialize a state store. Cannot be null.
      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 returns null, 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 with null key or value are ignored. Aggregating is a generalization of combining 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 specified Aggregator 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 the Initializer) 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 for cache size, and commit 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 in StreamsConfig via parameter APPLICATION_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 via Topology.describe().

      Type Parameters:
      VR - the value type of the resulting KTable
      Parameters:
      initializer - an Initializer that computes an initial intermediate aggregation result
      aggregator - an Aggregator 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 returns null, 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 with null key or value are ignored. Aggregating is a generalization of combining via reduce(...) as it, for example, allows the result to have a different type than the input values. 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.

      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 specified Aggregator 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 the Initializer) 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 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
       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)
       
      For non-local keys, a custom RPC mechanism must be implemented using 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().

      Type Parameters:
      VR - the value type of the resulting KTable
      Parameters:
      initializer - an Initializer that computes an initial intermediate aggregation result
      aggregator - an Aggregator that computes a new aggregate result
      materialized - an instance of Materialized used to materialize a state store. Cannot be null.
      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 with null key or value are ignored. Aggregating is a generalization of combining via reduce(...) as it, for example, allows the result to have a different type than the input values. 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.

      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 specified Aggregator 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 the Initializer) 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 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
       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)
       
      For non-local keys, a custom RPC mechanism must be implemented using 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().

      Type Parameters:
      VR - the value type of the resulting KTable
      Parameters:
      initializer - an Initializer that computes an initial intermediate aggregation result
      aggregator - an Aggregator that computes a new aggregate result
      named - a Named config used to name the processor in the topology
      materialized - an instance of Materialized used to materialize a state store. Cannot be null.
      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 returns null, 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

      <W extends Window> TimeWindowedKStream<K,V> windowedBy(Windows<W> windows)
      Create a new TimeWindowedKStream instance 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 TimeWindowedKStream
    • windowedBy

      TimeWindowedKStream<K,V> windowedBy(SlidingWindows windows)
      Create a new TimeWindowedKStream instance that can be used to perform sliding windowed aggregations.
      Parameters:
      windows - the specification of the aggregation SlidingWindows
      Returns:
      an instance of TimeWindowedKStream
    • windowedBy

      SessionWindowedKStream<K,V> windowedBy(SessionWindows windows)
      Create a new SessionWindowedKStream instance that can be used to perform session windowed aggregations.
      Parameters:
      windows - the specification of the aggregation SessionWindows
      Returns:
      an instance of TimeWindowedKStream
    • cogroup

      <VOut> CogroupedKStream<K,VOut> cogroup(Aggregator<? super K,? super V,VOut> aggregator)
      Create a new CogroupedKStream from the this grouped KStream to allow cogrouping other KGroupedStream to it. 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.

      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 CogroupedKStream.aggregate(Initializer)) and the record's value.

      Type Parameters:
      VOut - the type of the output values
      Parameters:
      aggregator - an Aggregator that computes a new aggregate result
      Returns:
      a CogroupedKStream