K - Type of keysV - Type of valuespublic interface SessionWindowedKStream<K,V>
SessionWindowedKStream is an abstraction of a windowed record stream of KeyValue pairs.
 It is an intermediate representation after a grouping and windowing of a KStream before an aggregation is
 applied to the new (partitioned) windows resulting in a windowed KTable (a KTable is a KTable with key type Windowed).
 
 SessionWindows are dynamic data driven windows.
 They have no fixed time boundaries, rather the size of the window is determined by the records.
 
 The result is written into a local SessionStore (which is basically an ever-updating
 materialized view) that can be queried using the name provided in the Materialized instance.
 Furthermore, updates to the store are sent downstream into a windowed KTable changelog stream, where
 "windowed" implies that the KTable key is a combined key of the original record key and a window ID.
 New events are added to sessions until their grace period ends (see SessionWindows.grace(Duration)).
 
 A SessionWindowedKStream must be obtained from a KGroupedStream via
 KGroupedStream.windowedBy(SessionWindows).
KStream, 
KGroupedStream, 
SessionWindows| Modifier and Type | Method and Description | 
|---|---|
| <VR> KTable<Windowed<K>,VR> | aggregate(Initializer<VR> initializer,
         Aggregator<? super K,? super V,VR> aggregator,
         Merger<? super K,VR> sessionMerger)Aggregate the values of records in this stream by the grouped key and defined sessions. | 
| <VR> KTable<Windowed<K>,VR> | aggregate(Initializer<VR> initializer,
         Aggregator<? super K,? super V,VR> aggregator,
         Merger<? super K,VR> sessionMerger,
         Materialized<K,VR,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)Aggregate the values of records in this stream by the grouped key and defined sessions. | 
| <VR> KTable<Windowed<K>,VR> | aggregate(Initializer<VR> initializer,
         Aggregator<? super K,? super V,VR> aggregator,
         Merger<? super K,VR> sessionMerger,
         Named named)Aggregate the values of records in this stream by the grouped key and defined sessions. | 
| <VR> KTable<Windowed<K>,VR> | aggregate(Initializer<VR> initializer,
         Aggregator<? super K,? super V,VR> aggregator,
         Merger<? super K,VR> sessionMerger,
         Named named,
         Materialized<K,VR,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)Aggregate the values of records in this stream by the grouped key and defined sessions. | 
| KTable<Windowed<K>,Long> | count()Count the number of records in this stream by the grouped key and defined sessions. | 
| KTable<Windowed<K>,Long> | count(Materialized<K,Long,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)Count the number of records in this stream by the grouped key and defined sessions. | 
| KTable<Windowed<K>,Long> | count(Named named)Count the number of records in this stream by the grouped key and defined sessions. | 
| KTable<Windowed<K>,Long> | count(Named named,
     Materialized<K,Long,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)Count the number of records in this stream by the grouped key and defined sessions. | 
| KTable<Windowed<K>,V> | reduce(Reducer<V> reducer)Combine the values of records in this stream by the grouped key and defined sessions. | 
| KTable<Windowed<K>,V> | reduce(Reducer<V> reducer,
      Materialized<K,V,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)Combine the values of records in this stream by the grouped key and defined sessions. | 
| KTable<Windowed<K>,V> | reduce(Reducer<V> reducer,
      Named named)Combine the values of records in this stream by the grouped key and defined sessions. | 
| KTable<Windowed<K>,V> | reduce(Reducer<V> reducer,
      Named named,
      Materialized<K,V,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)Combine the values of records in this stream by the grouped key and defined sessions. | 
KTable<Windowed<K>,Long> count()
null key or value are ignored.
 
 The result is written into a local SessionStore (which is basically an ever-updating materialized view).
 The default key serde from the config will be used for serializing the result.
 If a different serde is required then you should use count(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 session and 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 intervall.
 
 For failure and recovery the store 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 queriable through Interactive Queries.
 
 You can retrieve all generated internal topic names via Topology.describe().
KTable<Windowed<K>,Long> count(Named named)
null key or value are ignored.
 
 The result is written into a local SessionStore (which is basically an ever-updating materialized view).
 The default key serde from the config will be used for serializing the result.
 If a different serde is required then you should use count(Named, 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 session and 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 intervall.
 
 For failure and recovery the store 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 queriable through Interactive Queries.
 
 You can retrieve all generated internal topic names via Topology.describe().
KTable<Windowed<K>,Long> count(Materialized<K,Long,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
null key or value are ignored.
 
 The result is written into a local SessionStore (which is basically an ever-updating materialized view)
 that can be queried using the name provided with 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 will be used to deduplicate consecutive updates
 to the same window and key if caching is enabled on the Materialized instance.
 When caching is enabled 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 intervall.
 
 To query the local SessionStore it must be obtained via
 KafkaStreams#store(...):
 
 KafkaStreams streams = ... // compute sum
 Sting queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance
 ReadOnlySessionStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>ReadOnlySessionStore<String, Long>);
 String key = "some-key";
 KeyValueIterator<Windowed<String>, Long> sumForKeyForWindows = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
 KafkaStreams.allMetadata() to
 query the value of the key on a parallel running instance of your Kafka Streams application.
 
 For failure and recovery the store 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().
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 providedKTable that contains "update" records with unmodified keys and Long values
 that represent the latest (rolling) count (i.e., number of records) for each key per sessionKTable<Windowed<K>,Long> count(Named named, Materialized<K,Long,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
null key or value are ignored.
 
 The result is written into a local SessionStore (which is basically an ever-updating materialized view)
 that can be queried using the name provided with 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 will be used to deduplicate consecutive updates
 to the same window and key if caching is enabled on the Materialized instance.
 When caching is enabled 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 intervall.
 
 To query the local SessionStore it must be obtained via
 KafkaStreams#store(...):
 
 KafkaStreams streams = ... // compute sum
 Sting queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance
 ReadOnlySessionStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>ReadOnlySessionStore<String, Long>);
 String key = "some-key";
 KeyValueIterator<Windowed<String>, Long> sumForKeyForWindows = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
 KafkaStreams.allMetadata() to
 query the value of the key on a parallel running instance of your Kafka Streams application.
 
 For failure and recovery the store 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().
named - a Named config used to name the processor in the topology. Cannot be null.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 providedKTable that contains "update" records with unmodified keys and Long values
 that represent the latest (rolling) count (i.e., number of records) for each key per session<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Merger<? super K,VR> sessionMerger)
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 SessionStore (which is basically an ever-updating materialized view).
 Furthermore, updates to the store are sent downstream into a KTable changelog stream.
 
 The specified Initializer is applied directly before the first input record per session is processed to
 provide an initial intermediate aggregation result that is used to process the first record per session.
 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.
 The specified Merger is used to merge two existing sessions into one, i.e., when the windows overlap,
 they are merged into a single session and the old sessions are discarded.
 Thus, aggregate() can be used to compute aggregate functions like count (c.f. count()).
 
 The default key and value serde from the config will be used for serializing the result.
 If a different serde is required then you should use
 aggregate(Initializer, Aggregator, Merger, Materialized).
 
 Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
 the same window and 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 intervall.
 
 For failure and recovery the store 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 queriable through Interactive Queries.
 
 You can retrieve all generated internal topic names via Topology.describe().
VR - the value type of the resulting KTableinitializer - an Initializer that computes an initial intermediate aggregation result. Cannot be null.aggregator - an Aggregator that computes a new aggregate result. Cannot be null.sessionMerger - a Merger that combines two aggregation results. Cannot be null.KTable that contains "update" records with unmodified keys, and values that represent
 the latest (rolling) aggregate for each key per session<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Merger<? super K,VR> sessionMerger, Named named)
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 SessionStore (which is basically an ever-updating materialized view).
 Furthermore, updates to the store are sent downstream into a KTable changelog stream.
 
 The specified Initializer is applied directly before the first input record per session is processed to
 provide an initial intermediate aggregation result that is used to process the first record per session.
 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.
 The specified Merger is used to merge two existing sessions into one, i.e., when the windows overlap,
 they are merged into a single session and the old sessions are discarded.
 Thus, aggregate() can be used to compute aggregate functions like count (c.f. count()).
 
 The default key and value serde from the config will be used for serializing the result.
 If a different serde is required then you should use
 aggregate(Initializer, Aggregator, Merger, Named, Materialized).
 
 Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
 the same window and 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 intervall.
 
 For failure and recovery the store 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 queriable through Interactive Queries.
 
 You can retrieve all generated internal topic names via Topology.describe().
VR - the value type of the resulting KTableinitializer - an Initializer that computes an initial intermediate aggregation result. Cannot be null.aggregator - an Aggregator that computes a new aggregate result. Cannot be null.sessionMerger - a Merger that combines two aggregation results. Cannot be null.named - a Named config used to name the processor in the topology. Cannot be null.KTable that contains "update" records with unmodified keys, and values that represent
 the latest (rolling) aggregate for each key per session<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Merger<? super K,VR> sessionMerger, Materialized<K,VR,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
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 SessionStore (which is basically an ever-updating materialized view)
 that can be queried using the store name as provided with Materialized.
 Furthermore, updates to the store are sent downstream into a KTable changelog stream.
 
 The specified Initializer is applied directly before the first input record per session is processed to
 provide an initial intermediate aggregation result that is used to process the first record per session.
 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.
 The specified Merger is used to merge two existing sessions into one, i.e., when the windows overlap,
 they are merged into a single session and the old sessions are discarded.
 Thus, aggregate() 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 window and key if caching is enabled on the Materialized instance.
 When caching is enabled 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 intervall.
 
 To query the local SessionStore it must be obtained via
 KafkaStreams#store(...):
 
 KafkaStreams streams = ... // some windowed aggregation on value type double
 Sting queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance
 ReadOnlySessionStore<String, Long> sessionStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>sessionStore());
 String key = "some-key";
 KeyValueIterator<Windowed<String>, Long> aggForKeyForSession = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
 KafkaStreams.allMetadata() to
 query the value of the key on a parallel running instance of your Kafka Streams application.
 
 For failure and recovery the store 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().
VR - the value type of the resulting KTableinitializer - an Initializer that computes an initial intermediate aggregation result. Cannot be null.aggregator - an Aggregator that computes a new aggregate result. Cannot be null.sessionMerger - a Merger that combines two aggregation results. Cannot be null.materialized - a Materialized config 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 key per session<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Merger<? super K,VR> sessionMerger, Named named, Materialized<K,VR,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
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 SessionStore (which is basically an ever-updating materialized view)
 that can be queried using the store name as provided with Materialized.
 Furthermore, updates to the store are sent downstream into a KTable changelog stream.
 
 The specified Initializer is applied directly before the first input record per session is processed to
 provide an initial intermediate aggregation result that is used to process the first record per session.
 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.
 The specified Merger is used to merge two existing sessions into one, i.e., when the windows overlap,
 they are merged into a single session and the old sessions are discarded.
 Thus, aggregate() can be used to compute aggregate functions like count (c.f. count()).
 
 Not all updates might get sent downstream, as an internal cache will be used to deduplicate consecutive updates
 to the same window and key if caching is enabled on the Materialized instance.
 When caching is enabled 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 intervall.
 
 To query the local SessionStore it must be obtained via
 KafkaStreams#store(...):
 
 KafkaStreams streams = ... // some windowed aggregation on value type double
 Sting queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance
 ReadOnlySessionStore<String, Long> sessionStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>sessionStore());
 String key = "some-key";
 KeyValueIterator<Windowed<String>, Long> aggForKeyForSession = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
 KafkaStreams.allMetadata() to
 query the value of the key on a parallel running instance of your Kafka Streams application.
 
 For failure and recovery the store 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().
VR - the value type of the resulting KTableinitializer - an Initializer that computes an initial intermediate aggregation result. Cannot be null.aggregator - an Aggregator that computes a new aggregate result. Cannot be null.sessionMerger - a Merger that combines two aggregation results. Cannot be null.named - a Named config used to name the processor in the topology. Cannot be null.materialized - a Materialized config 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 key per sessionKTable<Windowed<K>,V> reduce(Reducer<V> reducer)
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, Merger)).
 The result is written into a local SessionStore (which is basically an ever-updating materialized view).
 Furthermore, updates to the store are sent downstream into a KTable changelog stream.
 The default key and value serde from the config will be used for serializing the result.
 If a different serde is required then you should use reduce(Reducer, Materialized) .
 
 The value of the first record per session initialized the session result.
 The specified Reducer is applied for each additional input record per session 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;
   }
 }
 reduce() 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 window and 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 intervall.
 
 For failure and recovery the store 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.
 
 You can retrieve all generated internal topic names via Topology.describe().
KTable<Windowed<K>,V> reduce(Reducer<V> reducer, Named named)
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, Merger)).
 The result is written into a local SessionStore (which is basically an ever-updating materialized view).
 Furthermore, updates to the store are sent downstream into a KTable changelog stream.
 The default key and value serde from the config will be used for serializing the result.
 If a different serde is required then you should use reduce(Reducer, Named, Materialized) .
 
 The value of the first record per session initialized the session result.
 The specified Reducer is applied for each additional input record per session 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;
   }
 }
 reduce() 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 window and 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 intervall.
 
 For failure and recovery the store 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.
 
 You can retrieve all generated internal topic names via Topology.describe().
reducer - a Reducer that computes a new aggregate result. Cannot be null.named - a Named config used to name the processor in the topology. Cannot be null.KTable that contains "update" records with unmodified keys, and values that represent
 the latest (rolling) aggregate for each key per sessionKTable<Windowed<K>,V> reduce(Reducer<V> reducer, Materialized<K,V,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
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, Merger)).
 The result is written into a local SessionStore (which is basically an ever-updating materialized view)
 that can be queried using the store name as provided with Materialized.
 Furthermore, updates to the store are sent downstream into a KTable changelog stream.
 
 The value of the first record per session initialized the session result.
 The specified Reducer is applied for each additional input record per session 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;
   }
 }
 reduce() can be used to compute aggregate functions like sum, min, or max.
 
 Not all updates might get sent downstream, as an internal cache will be used to deduplicate consecutive updates
 to the same window and key if caching is enabled on the Materialized instance.
 When caching is enabled 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 intervall.
 
 To query the local SessionStore it must be obtained via
 KafkaStreams#store(...):
 
 KafkaStreams streams = ... // compute sum
 Sting queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance
 ReadOnlySessionStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>ReadOnlySessionStore<String, Long>);
 String key = "some-key";
 KeyValueIterator<Windowed<String>, Long> sumForKeyForWindows = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
 KafkaStreams.allMetadata() to
 query the value of the key on a parallel running instance of your Kafka Streams application.
 
 For failure and recovery the store 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().
reducer - a Reducer that computes a new aggregate result. Cannot be null.materialized - a Materialized config 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 key per sessionKTable<Windowed<K>,V> reduce(Reducer<V> reducer, Named named, Materialized<K,V,SessionStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
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, Merger)).
 The result is written into a local SessionStore (which is basically an ever-updating materialized view)
 that can be queried using the store name as provided with Materialized.
 Furthermore, updates to the store are sent downstream into a KTable changelog stream.
 
 The value of the first record per session initialized the session result.
 The specified Reducer is applied for each additional input record per session 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;
   }
 }
 reduce() can be used to compute aggregate functions like sum, min, or max.
 
 Not all updates might get sent downstream, as an internal cache will be used to deduplicate consecutive updates
 to the same window and key if caching is enabled on the Materialized instance.
 When caching is enabled 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 intervall.
 
 To query the local SessionStore it must be obtained via
 KafkaStreams.store(StoreQueryParameters)  KafkaStreams#store(...)}:
 
 KafkaStreams streams = ... // compute sum
 Sting queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance
 ReadOnlySessionStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>ReadOnlySessionStore<String, Long>);
 String key = "some-key";
 KeyValueIterator<Windowed<String>, Long> sumForKeyForWindows = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
 KafkaStreams.allMetadata() to
 query the value of the key on a parallel running instance of your Kafka Streams application.
 
 For failure and recovery the store 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().
reducer - a Reducer that computes a new aggregate result. Cannot be null.named - a Named config used to name the processor in the topology. Cannot be null.materialized - a Materialized config 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 key per session