@ -16,11 +16,13 @@
@@ -16,11 +16,13 @@
* /
package org.apache.kafka.streams.kstream ;
import org.apache.kafka.common.serialization.Serde ;
import org.apache.kafka.common.utils.Bytes ;
import org.apache.kafka.streams.KafkaStreams ;
import org.apache.kafka.streams.KeyValue ;
import org.apache.kafka.streams.StreamsConfig ;
import org.apache.kafka.streams.state.KeyValueStore ;
import org.apache.kafka.streams.state.QueryableStoreType ;
import org.apache.kafka.streams.state.WindowStore ;
/ * *
* { @code WindowedKStream } is an abstraction of a < i > windowed < / i > record stream of { @link KeyValue } pairs .
@ -31,6 +33,14 @@ import org.apache.kafka.streams.state.KeyValueStore;
@@ -31,6 +33,14 @@ import org.apache.kafka.streams.state.KeyValueStore;
* new ( partitioned ) windows resulting in a windowed { @link KTable }
* ( a < emph > windowed < / emph > { @code KTable } is a { @link KTable } with key type { @link Windowed Windowed < K > } .
* < p >
* The specified { @code windows } define either hopping time windows that can be overlapping or tumbling ( c . f .
* { @link TimeWindows } ) or they define landmark windows ( c . f . { @link UnlimitedWindows } ) .
* The result is written into a local windowed { @link KeyValueStore } ( which is basically an ever - updating
* materialized view ) that can be queried using the name provided in the { @link Materialized } instance .
* Windows are retained until their retention time expires ( c . f . { @link Windows # until ( long ) } ) .
* Furthermore , updates to the store are sent downstream into a windowed { @link KTable } changelog stream , where
* "windowed" implies that the { @link KTable } key is a combined key of the original record key and a window ID .
* A { @code WindowedKStream } must be obtained from a { @link KGroupedStream } via { @link KGroupedStream # windowedBy ( Windows ) } .
*
* @param < K > Type of keys
@ -43,13 +53,6 @@ public interface WindowedKStream<K, V> {
@@ -43,13 +53,6 @@ public interface WindowedKStream<K, V> {
/ * *
* Count the number of records in this stream by the grouped key and the defined windows .
* Records with { @code null } key or value are ignored .
* The specified { @code windows } define either hopping time windows that can be overlapping or tumbling ( c . f .
* { @link TimeWindows } ) or they define landmark windows ( c . f . { @link UnlimitedWindows } ) .
* The result is written into a local windowed { @link KeyValueStore } ( which is basically an ever - updating
* materialized view ) that can be queried using the provided { @code queryableName } .
* Windows are retained until their retention time expires ( c . f . { @link Windows # until ( long ) } ) .
* Furthermore , updates to the store are sent downstream into a windowed { @link KTable } changelog stream , where
* "windowed" implies that the { @link KTable } key is a combined key of the original record key and a window ID .
* < p >
* Not all updates might get sent downstream , as an internal cache is used to deduplicate consecutive updates to
* the same window and key .
@ -70,6 +73,38 @@ public interface WindowedKStream<K, V> {
@@ -70,6 +73,38 @@ public interface WindowedKStream<K, V> {
* /
KTable < Windowed < K > , Long > count ( ) ;
/ * *
* Count the number of records in this stream by the grouped key and the defined windows .
* Records with { @code null } key or value are ignored .
* < p >
* 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 { @link 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 { @link StreamsConfig configuration } parameters for
* { @link StreamsConfig # CACHE_MAX_BYTES_BUFFERING_CONFIG cache size } , and
* { @link StreamsConfig # COMMIT_INTERVAL_MS_CONFIG commit intervall }
*
* < p >
* To query the local windowed { @link KeyValueStore } it must be obtained via
* { @link KafkaStreams # store ( String , QueryableStoreType ) KafkaStreams # store ( . . . ) } :
* < pre > { @code
* 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 > countForWordsForWindows = localWindowStore . fetch ( key , timeFrom , timeTo ) ; // key must be local (application state is shared over all running Kafka Streams instances)
* } < / pre >
* For non - local keys , a custom RPC mechanism must be implemented using { @link KafkaStreams # allMetadata ( ) } to
* query the value of the key on a parallel running instance of your Kafka Streams application .
*
* @return a { @link KTable } that contains "update" records with unmodified keys and { @link Long } values that
* represent the latest ( rolling ) count ( i . e . , number of records ) for each key
* /
KTable < Windowed < K > , Long > count ( final Materialized < K , Long , WindowStore < Bytes , byte [ ] > > materialized ) ;
/ * *
* Aggregate the values of records in this stream by the grouped key .
* Records with { @code null } key or value are ignored .
@ -84,9 +119,12 @@ public interface WindowedKStream<K, V> {
@@ -84,9 +119,12 @@ public interface WindowedKStream<K, V> {
* The specified { @link 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
* { @link Initializer } ) and the record ' s value .
* Thus , { @code aggregate ( Initializer , Aggregator , Serde ) } can be used to compute aggregate functions like
* Thus , { @code aggregate ( Initializer , Aggregator ) } can be used to compute aggregate functions like
* count ( c . f . { @link # count ( ) } ) .
* < p >
* The default value serde from config will be used for serializing the result .
* If a different serde is required then you should use { @link # aggregate ( Initializer , Aggregator , Materialized ) } .
* < p >
* 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
@ -102,17 +140,63 @@ public interface WindowedKStream<K, V> {
@@ -102,17 +140,63 @@ public interface WindowedKStream<K, V> {
* Note that the internal store name may not be queriable through Interactive Queries .
* You can retrieve all generated internal topic names via { @link KafkaStreams # toString ( ) } .
*
* @param < VR > the value type of the resulting { @link KTable }
* @param initializer an { @link Initializer } that computes an initial intermediate aggregation result
* @param aggregator an { @link Aggregator } that computes a new aggregate result
* @param aggValueSerde aggregate value serdes for materializing the aggregated table ,
* if not specified the default serdes defined in the configs will be used
* @return a { @link KTable } that contains "update" records with unmodified keys , and values that represent the
* latest ( rolling ) aggregate for each key
* /
< VR > KTable < Windowed < K > , VR > aggregate ( final Initializer < VR > initializer ,
final Aggregator < ? super K , ? super V , VR > aggregator ) ;
/ * *
* Aggregate the values of records in this stream by the grouped key .
* Records with { @code null } key or value are ignored .
* Aggregating is a generalization of { @link # reduce ( Reducer ) 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 { @link KeyValueStore } ( which is basically an ever - updating materialized view )
* that can be queried using the store name as provided with { @link Materialized } .
* < p >
* The specified { @link 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 { @link 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
* { @link Initializer } ) and the record ' s value .
* Thus , { @code aggregate ( Initializer , Aggregator , Materialized ) } can be used to compute aggregate functions like
* count ( c . f . { @link # count ( ) } ) .
* < p >
* < p >
* 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 { @link Materialized } instance .
* 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 the { @link StreamsConfig configuration } parameters for
* { @link StreamsConfig # CACHE_MAX_BYTES_BUFFERING_CONFIG cache size } , and
* { @link StreamsConfig # COMMIT_INTERVAL_MS_CONFIG commit intervall }
*
* < p >
* To query the local windowed { @link KeyValueStore } it must be obtained via
* { @link KafkaStreams # store ( String , QueryableStoreType ) KafkaStreams # store ( . . . ) } :
* < pre > { @code
* 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)
* } < / pre >
*
* @param initializer an { @link Initializer } that computes an initial intermediate aggregation result
* @param aggregator an { @link Aggregator } that computes a new aggregate result
* @param materialized an instance of { @link Materialized } used to materialize a state store . Cannot be { @code null } .
* @param < VR > the value type of the resulting { @link KTable }
* @return a { @link KTable } that contains "update" records with unmodified keys , and values that represent the
* latest ( rolling ) aggregate for each key
* /
< VR > KTable < Windowed < K > , VR > aggregate ( final Initializer < VR > initializer ,
final Aggregator < ? super K , ? super V , VR > aggregator ,
final Serde < VR > aggValueSerde ) ;
final Materialized < K , VR , WindowStore < Bytes , byte [ ] > > materialized ) ;
/ * *
* Combine the values of records in this stream by the grouped key .
@ -147,4 +231,46 @@ public interface WindowedKStream<K, V> {
@@ -147,4 +231,46 @@ public interface WindowedKStream<K, V> {
* latest ( rolling ) aggregate for each key
* /
KTable < Windowed < K > , V > reduce ( final Reducer < V > reducer ) ;
/ * *
* Combine the values of records in this stream by the grouped key .
* Records with { @code 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 .
* The result is written into a local { @link KeyValueStore } ( which is basically an ever - updating materialized view )
* that can be queried using the store name as provided with { @link Materialized } .
* Furthermore , updates to the store are sent downstream into a { @link KTable } changelog stream .
* < p >
* The specified { @link 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 { @link Reducer } is not applied and the new aggregate will be the record ' s
* value as - is .
* Thus , { @code reduce ( Reducer , String ) } can be used to compute aggregate functions like sum , min , or max .
* < p >
* 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 { @link Materialized } instance .
* 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 the { @link StreamsConfig configuration } parameters for
* { @link StreamsConfig # CACHE_MAX_BYTES_BUFFERING_CONFIG cache size } , and
* { @link StreamsConfig # COMMIT_INTERVAL_MS_CONFIG commit intervall }
* < p >
* To query the local windowed { @link KeyValueStore } it must be obtained via
* { @link KafkaStreams # store ( String , QueryableStoreType ) KafkaStreams # store ( . . . ) } :
* < pre > { @code
* 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)
* } < / pre >
*
*
* @param reducer a { @link Reducer } that computes a new aggregate result
* @return a { @link KTable } that contains "update" records with unmodified keys , and values that represent the
* latest ( rolling ) aggregate for each key
* /
KTable < Windowed < K > , V > reduce ( final Reducer < V > reducer ,
final Materialized < K , V , WindowStore < Bytes , byte [ ] > > materialized ) ;
}