Interface TimeWindowedKStream<K,V> 
- Type Parameters:
- K- Type of keys
- V- Type of values
TimeWindowedKStream 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).
 
 The specified windows define either hopping time windows that can be overlapping or tumbling (c.f.
 TimeWindows) or they define landmark windows (c.f. UnlimitedWindows).
 
 The result is written into a local WindowStore (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 TimeWindows until their grace period ends (see TimeWindows.grace(Duration)).
 
 A TimeWindowedKStream must be obtained from a KGroupedStream via
 KGroupedStream.windowedBy(Windows).
- See Also:
- 
Method SummaryModifier and TypeMethodDescriptionaggregate(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator) Aggregate the values of records in this stream by the grouped key and defined windows.aggregate(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Materialized<K, VR, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in this stream by the grouped key and defined windows.aggregate(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Named named) Aggregate the values of records in this stream by the grouped key and defined windows.aggregate(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Named named, Materialized<K, VR, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in this stream by the grouped key and defined windows.count()Count the number of records in this stream by the grouped key and defined windows.count(Materialized<K, Long, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Count the number of records in this stream by the grouped key and defined windows.Count the number of records in this stream by the grouped key and defined windows.count(Named named, Materialized<K, Long, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Count the number of records in this stream by the grouped key and defined windows.emitStrategy(EmitStrategy emitStrategy) Configure when the aggregated result will be emitted forTimeWindowedKStream.Combine the values of records in this stream by the grouped key and defined windows.reduce(Reducer<V> reducer, Materialized<K, V, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Combine the values of records in this stream by the grouped key and defined windows.Combine the values of records in this stream by the grouped key and defined windows.reduce(Reducer<V> reducer, Named named, Materialized<K, V, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Combine the values of records in this stream by the grouped key and defined windows.
- 
Method Details- 
countCount the number of records in this stream by the grouped key and defined windows. Records withnullkey or value are ignored.The result is written into a local WindowStore(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 usecount(Materialized). Furthermore, updates to the store are sent downstream into aKTablechangelog stream. 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 theconfigurationparameters forcache size, andcommit interval.For failure and recovery the store (which always will be of type TimestampedWindowStore) 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 inStreamsConfigvia parameterAPPLICATION_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().
- 
countCount the number of records in this stream by the grouped key and defined windows. Records withnullkey or value are ignored.The result is written into a local WindowStore(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 usecount(Named, Materialized). Furthermore, updates to the store are sent downstream into aKTablechangelog stream. 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 theconfigurationparameters forcache size, andcommit interval.For failure and recovery the store (which always will be of type TimestampedWindowStore) 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 inStreamsConfigvia parameterAPPLICATION_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().
- 
countKTable<Windowed<K>,Long> count(Materialized<K, Long, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Count the number of records in this stream by the grouped key and defined windows. Records withnullkey or value are ignored.The result is written into a local WindowStore(which is basically an ever-updating materialized view) that can be queried using the name provided withMaterialized. Furthermore, updates to the store are sent downstream into aKTablechangelog 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 Materializedinstance. 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 theconfigurationparameters forcache size, andcommit interval.To query the local ReadOnlyWindowStoreit must be obtained viaKafkaStreams#store(...):
 For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // counting words Store queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance StoreQueryParameters<ReadOnlyKeyValueStore<K, ValueAndTimestamp<Long>>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedWindowStore()); ReadOnlyWindowStore<K, ValueAndTimestamp<Long>> localWindowStore = streams.store(storeQueryParams); K key = "some-word"; long fromTime = ...; long toTime = ...; WindowStoreIterator<ValueAndTimestamp<Long>> countForWordsForWindows = localWindowStore.fetch(key, timeFrom, timeTo); // 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 TimestampedWindowStore-- 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:
- materialized- an instance of- Materializedused 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 windowed KTablethat contains "update" records with unmodified keys andLongvalues that represent the latest (rolling) count (i.e., number of records) for each key within a window
 
- 
countKTable<Windowed<K>,Long> count(Named named, Materialized<K, Long, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Count the number of records in this stream by the grouped key and defined windows. Records withnullkey or value are ignored.The result is written into a local WindowStore(which is basically an ever-updating materialized view) that can be queried using the name provided withMaterialized. Furthermore, updates to the store are sent downstream into aKTablechangelog 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 Materializedinstance. 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 theconfigurationparameters forcache size, andcommit intervalTo query the local ReadOnlyWindowStoreit must be obtained viaKafkaStreams#store(...):
 For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // counting words Store queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance StoreQueryParameters<ReadOnlyKeyValueStore<K, ValueAndTimestamp<Long>>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedWindowStore()); ReadOnlyWindowStore<K, ValueAndTimestamp<Long>> localWindowStore = streams.store(storeQueryParams); K key = "some-word"; long fromTime = ...; long toTime = ...; WindowStoreIterator<ValueAndTimestamp<Long>> countForWordsForWindows = localWindowStore.fetch(key, timeFrom, timeTo); // 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 TimestampedWindowStore-- 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:
- named- a- Namedconfig used to name the processor in the topology. Cannot be- null.
- materialized- an instance of- Materializedused 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 windowed KTablethat contains "update" records with unmodified keys andLongvalues that represent the latest (rolling) count (i.e., number of records) for each key within a window
 
- 
aggregate<VR> KTable<Windowed<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 and defined windows. Records withnullkey or value are ignored. Aggregating is a generalization ofcombining via reduce(...)as it, for example, allows the result to have a different type than the input values. The result is written into a localWindowStore(which is basically an ever-updating materialized view). Furthermore, updates to the store are sent downstream into aKTablechangelog stream.The specified Initializeris applied directly before the first input record (per key) in each window is processed to provide an initial intermediate aggregation result that is used to process the first record for the window (per key). The specifiedAggregatoris 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 theInitializer) and the record's value. 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, 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 theconfigurationparameters forcache size, andcommit interval.For failure and recovery the store (which always will be of type TimestampedWindowStore) 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 inStreamsConfigvia parameterAPPLICATION_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- Initializerthat computes an initial intermediate aggregation result. Cannot be- null.
- aggregator- an- Aggregatorthat computes a new aggregate result. Cannot be- null.
- Returns:
- a windowed KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key within a window
 
- 
aggregate<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Named named) Aggregate the values of records in this stream by the grouped key and defined windows. Records withnullkey or value are ignored. Aggregating is a generalization ofcombining via reduce(...)as it, for example, allows the result to have a different type than the input values. The result is written into a localWindowStore(which is basically an ever-updating materialized view). Furthermore, updates to the store are sent downstream into aKTablechangelog stream.The specified Initializeris applied directly before the first input record (per key) in each window is processed to provide an initial intermediate aggregation result that is used to process the first record for the window (per key). The specifiedAggregatoris 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 theInitializer) and the record's value. 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, 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 theconfigurationparameters forcache size, andcommit interval.For failure and recovery the store (which always will be of type TimestampedWindowStore) 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 inStreamsConfigvia parameterAPPLICATION_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- Initializerthat computes an initial intermediate aggregation result. Cannot be- null.
- aggregator- an- Aggregatorthat computes a new aggregate result. Cannot be- null.
- named- a- Namedconfig used to name the processor in the topology. Cannot be- null.
- Returns:
- a windowed KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key within a window
 
- 
aggregate<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Materialized<K, VR, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in this stream by the grouped key and defined windows. Records withnullkey or value are ignored. Aggregating is a generalization ofcombining via reduce(...)as it, for example, allows the result to have a different type than the input values. The result is written into a localWindowStore(which is basically an ever-updating materialized view) that can be queried using the store name as provided withMaterialized. Furthermore, updates to the store are sent downstream into aKTablechangelog stream.The specified Initializeris applied directly before the first input record (per key) in each window is processed to provide an initial intermediate aggregation result that is used to process the first record for the window (per key). The specifiedAggregatoris 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 theInitializer) and the record's value. 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 Materializedinstance. 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 theconfigurationparameters forcache size, andcommit interval.To query the local ReadOnlyWindowStoreit must be obtained viaKafkaStreams#store(...):
 For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // counting words Store queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance StoreQueryParameters<ReadOnlyKeyValueStore<K, ValueAndTimestamp<VR>>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedWindowStore()); ReadOnlyWindowStore<K, ValueAndTimestamp<VR>> localWindowStore = streams.store(storeQueryParams); K key = "some-word"; long fromTime = ...; long toTime = ...; WindowStoreIterator<ValueAndTimestamp<VR>> aggregateStore = localWindowStore.fetch(key, timeFrom, timeTo); // 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 TimestampedWindowStore-- 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 theMaterializedinstance 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().- Type Parameters:
- VR- the value type of the resulting- KTable
- Parameters:
- initializer- an- Initializerthat computes an initial intermediate aggregation result. Cannot be- null.
- aggregator- an- Aggregatorthat computes a new aggregate result. Cannot be- null.
- materialized- a- Materializedconfig used to materialize a state store. Cannot be- null.
- Returns:
- a windowed KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key within a window
 
- 
aggregate<VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K, ? super V, VR> aggregator, Named named, Materialized<K, VR, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in this stream by the grouped key and defined windows. Records withnullkey or value are ignored. Aggregating is a generalization ofcombining via reduce(...)as it, for example, allows the result to have a different type than the input values. The result is written into a localWindowStore(which is basically an ever-updating materialized view) that can be queried using the store name as provided withMaterialized. Furthermore, updates to the store are sent downstream into aKTablechangelog stream.The specified Initializeris applied directly before the first input record (per key) in each window is processed to provide an initial intermediate aggregation result that is used to process the first record for the window (per key). The specifiedAggregatoris 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 theInitializer) and the record's value. 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 Materializedinstance. 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 theconfigurationparameters forcache size, andcommit intervalTo query the local ReadOnlyWindowStoreit must be obtained viaKafkaStreams#store(...):
 For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // counting words Store queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance StoreQueryParameters<ReadOnlyKeyValueStore<K, ValueAndTimestamp<VR>>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedWindowStore()); ReadOnlyWindowStore<K, ValueAndTimestamp<VR>> localWindowStore = streams.store(storeQueryParams); K key = "some-word"; long fromTime = ...; long toTime = ...; WindowStoreIterator<ValueAndTimestamp<VR>> aggregateStore = localWindowStore.fetch(key, timeFrom, timeTo); // 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 TimestampedWindowStore-- 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 theMaterializedinstance 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().- Type Parameters:
- VR- the value type of the resulting- KTable
- Parameters:
- initializer- an- Initializerthat computes an initial intermediate aggregation result. Cannot be- null.
- aggregator- an- Aggregatorthat computes a new aggregate result. Cannot be- null.
- named- a- Namedconfig used to name the processor in the topology. Cannot be- null.
- materialized- a- Materializedconfig used to materialize a state store. Cannot be- null.
- Returns:
- a windowed KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key within a window
 
- 
reduceCombine the values of records in this stream by the grouped key and defined windows. Records withnullkey 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 result is written into a local WindowStore(which is basically an ever-updating materialized view). Furthermore, updates to the store are sent downstream into aKTablechangelog 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 usereduce(Reducer, Materialized).The value of the first record per window initialized the aggregation result. The specified Reduceris applied for each additional input record per window and computes a new aggregate using the current aggregate (first argument) and the record's value (second argument):
 Thus,// 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 configurationparameters forcache size, andcommit interval.For failure and recovery the store (which always will be of type TimestampedWindowStore) 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 inStreamsConfigvia parameterAPPLICATION_ID_CONFIG, "internalStoreName" is an internal name and "-changelog" is a fixed suffix.You can retrieve all generated internal topic names via Topology.describe().
- 
reduceCombine the values of records in this stream by the grouped key and defined windows. Records withnullkey or value are ignored. Combining implies that the type of the aggregate result is the same as the type of the input value.The result is written into a local WindowStore(which is basically an ever-updating materialized view). Furthermore, updates to the store are sent downstream into aKTablechangelog 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 usereduce(Reducer, Named, Materialized).The value of the first record per window initialized the aggregation result. The specified Reduceris applied for each additional input record per window and computes a new aggregate using the current aggregate (first argument) and the record's value (second argument):
 Thus,// 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 configurationparameters forcache size, andcommit interval.For failure and recovery the store (which always will be of type TimestampedWindowStore) 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 inStreamsConfigvia parameterAPPLICATION_ID_CONFIG, "internalStoreName" is an internal name and "-changelog" is a fixed suffix.You can retrieve all generated internal topic names via Topology.describe().- Parameters:
- reducer- a- Reducerthat computes a new aggregate result. Cannot be- null.
- named- a- Namedconfig used to name the processor in the topology. Cannot be- null.
- Returns:
- a windowed KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key within a window
 
- 
reduceKTable<Windowed<K>,V> reduce(Reducer<V> reducer, Materialized<K, V, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Combine the values of records in this stream by the grouped key and defined windows. Records withnullkey or value are ignored. Combining implies that the type of the aggregate result is the same as the type of the input value.The result is written into a local WindowStore(which is basically an ever-updating materialized view) that can be queried using the store name as provided withMaterialized. Furthermore, updates to the store are sent downstream into aKTablechangelog stream.The value of the first record per window initialized the aggregation result. The specified Reduceris applied for each additional input record per window and computes a new aggregate using the current aggregate (first argument) and the record's value (second argument):
 Thus,// 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 Materializedinstance. 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 theconfigurationparameters forcache size, andcommit interval.To query the local ReadOnlyWindowStoreit must be obtained viaKafkaStreams#store(...):
 For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // counting words Store queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance StoreQueryParameters<ReadOnlyKeyValueStore<K, ValueAndTimestamp<V>>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedWindowStore()); ReadOnlyWindowStore<K, ValueAndTimestamp<V>> localWindowStore = streams.store(storeQueryParams); K key = "some-word"; long fromTime = ...; long toTime = ...; WindowStoreIterator<ValueAndTimestamp<V>> reduceStore = localWindowStore.fetch(key, timeFrom, timeTo); // 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 TimestampedWindowStore-- 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:
- reducer- a- Reducerthat computes a new aggregate result. Cannot be- null.
- materialized- a- Materializedconfig used to materialize a state store. Cannot be- null.
- Returns:
- a windowed KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key within a window
 
- 
reduceKTable<Windowed<K>,V> reduce(Reducer<V> reducer, Named named, Materialized<K, V, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Combine the values of records in this stream by the grouped key and defined windows. Records withnullkey or value are ignored. Combining implies that the type of the aggregate result is the same as the type of the input value.The result is written into a local WindowStore(which is basically an ever-updating materialized view) that can be queried using the store name as provided withMaterialized. Furthermore, updates to the store are sent downstream into aKTablechangelog stream.The value of the first record per window initialized the aggregation result. The specified Reduceris applied for each additional input record per window and computes a new aggregate using the current aggregate (first argument) and the record's value (second argument):
 Thus,// 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 Materializedinstance. 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 theconfigurationparameters forcache size, andcommit interval.To query the local ReadOnlyWindowStoreit must be obtained viaKafkaStreams#store(...):
 For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // counting words Store queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance StoreQueryParameters<ReadOnlyKeyValueStore<K, ValueAndTimestamp<V>>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedWindowStore()); ReadOnlyWindowStore<K, ValueAndTimestamp<V>> localWindowStore = streams.store(storeQueryParams); K key = "some-word"; long fromTime = ...; long toTime = ...; WindowStoreIterator<ValueAndTimestamp<V>> reduceStore = localWindowStore.fetch(key, timeFrom, timeTo); // 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 TimestampedWindowStore-- 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:
- reducer- a- Reducerthat computes a new aggregate result. Cannot be- null.
- named- a- Namedconfig used to name the processor in the topology. Cannot be- null.
- materialized- a- Materializedconfig used to materialize a state store. Cannot be- null.
- Returns:
- a windowed KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key within a window
 
- 
emitStrategyConfigure when the aggregated result will be emitted forTimeWindowedKStream.For example, for EmitStrategy.onWindowClose()strategy, the aggregated result for a window will only be emitted when the window closes. ForEmitStrategy.onWindowUpdate()strategy, the aggregated result for a window will be emitted whenever there is an update to the window. Note that whether the result will be available in downstream also depends on cache policy.- Parameters:
- emitStrategy-- EmitStrategyto configure when the aggregated result for a window will be emitted.
- Returns:
- a TimeWindowedKStreamwithEmitStrategyconfigured.
 
 
-