Interface TimeWindowedKStream<K,V>
-
- Type Parameters:
K- Type of keysV- Type of values
public interface TimeWindowedKStream<K,V>TimeWindowedKStreamis an abstraction of a windowed record stream ofKeyValuepairs. It is an intermediate representation of aKStreamin order to apply a windowed aggregation operation on the originalKStreamrecords.It is an intermediate representation after a grouping and windowing of a
KStreambefore an aggregation is applied to the new (partitioned) windows resulting in a windowedKTable(awindowed KTableis aKTablewith key typeWindowed.The specified
windowsdefine 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 windowedKeyValueStore(which is basically an ever-updating materialized view) that can be queried using the name provided in theMaterializedinstance. Windows are retained until their retention time expires (c.f.Windows.until(long)). Furthermore, updates to the store are sent downstream into a windowedKTablechangelog stream, where "windowed" implies that theKTablekey is a combined key of the original record key and a window ID. AWindowedKStreammust be obtained from aKGroupedStreamviaKGroupedStream.windowedBy(Windows).- See Also:
KStream,KGroupedStream
-
-
Method Summary
All Methods Instance Methods Abstract Methods Modifier and Type Method Description <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.<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.KTable<Windowed<K>,java.lang.Long>count()Count the number of records in this stream by the grouped key and the defined windows.KTable<Windowed<K>,java.lang.Long>count(Materialized<K,java.lang.Long,WindowStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)Count the number of records in this stream by the grouped key and the defined windows.KTable<Windowed<K>,V>reduce(Reducer<V> reducer)Combine the values of records in this stream by the grouped key.KTable<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.
-
-
-
Method Detail
-
count
KTable<Windowed<K>,java.lang.Long> count()
Count the number of records in this stream by the grouped key and the defined windows. Records withnullkey or value are ignored.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 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
StreamsConfigvia parameterAPPLICATION_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 viaKafkaStreams.toString().- Returns:
- a
KTablethat contains "update" records with unmodified keys andLongvalues that represent the latest (rolling) count (i.e., number of records) for each key
-
count
KTable<Windowed<K>,java.lang.Long> count(Materialized<K,java.lang.Long,WindowStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Count the number of records in this stream by the grouped key and the defined windows. Records withnullkey or value are ignored.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 intervallTo query the local windowed
KeyValueStoreit 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 ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore()); String key = "some-word"; long fromTime = ...; long toTime = ...; WindowStoreIterator<Long> countForWordsForWindows = localWindowStore.fetch(key, timeFrom, timeTo); // 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.- Parameters:
materialized- an instance ofMaterializedused to materialize a state store. Cannot benull. Note: the valueSerde will be automatically set toSerdes.Long()if there is no valueSerde provided- Returns:
- a
KTablethat contains "update" records with unmodified keys andLongvalues that represent the latest (rolling) count (i.e., number of records) for each key
-
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. 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 localKeyValueStore(which is basically an ever-updating materialized view) that can be queried using the providedqueryableStoreName. Furthermore, updates to the store are sent downstream into aKTablechangelog stream.The specified
Initializeris 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 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(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
configurationparameters forcache size, andcommit 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
StreamsConfigvia parameterAPPLICATION_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 viaKafkaStreams.toString().- Type Parameters:
VR- the value type of the resultingKTable- Parameters:
initializer- anInitializerthat computes an initial intermediate aggregation resultaggregator- anAggregatorthat computes a new aggregate result- Returns:
- a
KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key
-
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. 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 localKeyValueStore(which is basically an ever-updating materialized view) that can be queried using the store name as provided withMaterialized.The specified
Initializeris 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 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(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 will be used to deduplicate consecutive updates to the same window and key if caching is enabled on the
Materializedinstance. When caching is enable 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 intervallTo query the local windowed
KeyValueStoreit must be obtained viaKafkaStreams#store(...):KafkaStreams streams = ... // counting words Store queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore()); String key = "some-word"; long fromTime = ...; long toTime = ...; WindowStoreIterator<Long> aggregateStore = localWindowStore.fetch(key, timeFrom, timeTo); // key must be local (application state is shared over all running Kafka Streams instances)- Type Parameters:
VR- the value type of the resultingKTable- Parameters:
initializer- anInitializerthat computes an initial intermediate aggregation resultaggregator- anAggregatorthat computes a new aggregate resultmaterialized- an instance ofMaterializedused to materialize a state store. Cannot benull.- Returns:
- a
KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key
-
reduce
KTable<Windowed<K>,V> reduce(Reducer<V> reducer)
Combine the values of records in this stream by the grouped key. 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 localKeyValueStore(which is basically an ever-updating materialized view) that can be queried using the providedqueryableStoreName. Furthermore, updates to the store are sent downstream into aKTablechangelog stream.The specified
Reduceris 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 theReduceris not applied and the new aggregate will be the record's value as-is. Thus,reduce(Reducer, String)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
configurationparameters forcache size, andcommit 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
StreamsConfigvia parameterAPPLICATION_ID_CONFIG, "internalStoreName" is an internal name and "-changelog" is a fixed suffix. You can retrieve all generated internal topic names viaKafkaStreams.toString().
-
reduce
KTable<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. 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 localKeyValueStore(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
Reduceris 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 theReduceris not applied and the new aggregate will be the record's value as-is. Thus,reduce(Reducer, String)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 enable 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 intervallTo query the local windowed
KeyValueStoreit must be obtained viaKafkaStreams#store(...):KafkaStreams streams = ... // counting words Store queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<String, Long>windowStore()); String key = "some-word"; long fromTime = ...; long toTime = ...; WindowStoreIterator<Long> reduceStore = localWindowStore.fetch(key, timeFrom, timeTo); // key must be local (application state is shared over all running Kafka Streams instances)
-
-