@ -18,6 +18,7 @@ package org.apache.kafka.streams.kstream;
import org.apache.kafka.common.annotation.InterfaceStability ;
import org.apache.kafka.common.annotation.InterfaceStability ;
import org.apache.kafka.common.serialization.Serde ;
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.KafkaStreams ;
import org.apache.kafka.streams.KeyValue ;
import org.apache.kafka.streams.KeyValue ;
import org.apache.kafka.streams.StreamsConfig ;
import org.apache.kafka.streams.StreamsConfig ;
@ -145,6 +146,38 @@ public interface KGroupedStream<K, V> {
* /
* /
KTable < K , Long > count ( final StateStoreSupplier < KeyValueStore > storeSupplier ) ;
KTable < K , Long > count ( final StateStoreSupplier < KeyValueStore > storeSupplier ) ;
/ * *
* Count the number of records in this stream by the grouped key .
* Records with { @code null } key or value are ignored .
* The result is written into a local { @link KeyValueStore } ( which is basically an ever - updating materialized view )
* provided by the given { @code storeSupplier } .
* Furthermore , updates to the store are sent downstream into a { @link KTable } changelog stream .
* < 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
* 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 { @link KeyValueStore } it must be obtained via
* { @link KafkaStreams # store ( String , QueryableStoreType ) KafkaStreams # store ( . . . ) } .
* < pre > { @code
* KafkaStreams streams = . . . // counting words
* String queryableStoreName = "count-store" ; // the queryableStoreName should be the name of the store as defined by the Materialized instance
* ReadOnlyKeyValueStore < String , Long > localStore = streams . store ( queryableStoreName , QueryableStoreTypes . < String , Long > keyValueStore ( ) ) ;
* String key = "some-word" ;
* Long countForWord = localStore . get ( key ) ; // 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 .
*
* @param materialized an instance of { @link Materialized } used to materialize a state store . Cannot be { @code null } .
* @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 < K , Long > count ( final Materialized < K , Long , KeyValueStore < Bytes , byte [ ] > > materialized ) ;
/ * *
/ * *
* Count the number of records in this stream by the grouped key and the defined windows .
* 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 .
* Records with { @code null } key or value are ignored .
@ -395,7 +428,7 @@ public interface KGroupedStream<K, V> {
* and "-changelog" is a fixed suffix .
* and "-changelog" is a fixed suffix .
* You can retrieve all generated internal topic names via { @link KafkaStreams # toString ( ) } .
* You can retrieve all generated internal topic names via { @link KafkaStreams # toString ( ) } .
*
*
* @param reducer a { @link Reducer } that computes a new aggregate result
* @param reducer a { @link Reducer } that computes a new aggregate result . Cannot be { @code null } .
* @return a { @link KTable } that contains "update" records with unmodified keys , and values that represent the
* @return a { @link KTable } that contains "update" records with unmodified keys , and values that represent the
* latest ( rolling ) aggregate for each key
* latest ( rolling ) aggregate for each key
* /
* /
@ -452,12 +485,14 @@ public interface KGroupedStream<K, V> {
* provide { @code queryableStoreName } , and "-changelog" is a fixed suffix .
* provide { @code queryableStoreName } , and "-changelog" is a fixed suffix .
* You can retrieve all generated internal topic names via { @link KafkaStreams # toString ( ) } .
* You can retrieve all generated internal topic names via { @link KafkaStreams # toString ( ) } .
*
*
* @param reducer a { @link Reducer } that computes a new aggregate result
* @param reducer a { @link Reducer } that computes a new aggregate result . Cannot be { @code null } .
* @param queryableStoreName the name of the underlying { @link KTable } state store ; valid characters are ASCII
* @param queryableStoreName the name of the underlying { @link KTable } state store ; valid characters are ASCII
* alphanumerics , '.' , '_' and '-' . If { @code null } then this will be equivalent to { @link KGroupedStream # reduce ( Reducer ) } ( ) } .
* alphanumerics , '.' , '_' and '-' . If { @code null } then this will be equivalent to { @link KGroupedStream # reduce ( Reducer ) } ( ) } .
* @return a { @link KTable } that contains "update" records with unmodified keys , and values that represent the
* @return a { @link KTable } that contains "update" records with unmodified keys , and values that represent the
* latest ( rolling ) aggregate for each key
* latest ( rolling ) aggregate for each key
* @deprectated use { @link # reduce ( Reducer , Materialized ) }
* /
* /
@Deprecated
KTable < K , V > reduce ( final Reducer < V > reducer ,
KTable < K , V > reduce ( final Reducer < V > reducer ,
final String queryableStoreName ) ;
final String queryableStoreName ) ;
@ -507,14 +542,68 @@ public interface KGroupedStream<K, V> {
* For non - local keys , a custom RPC mechanism must be implemented using { @link KafkaStreams # allMetadata ( ) } to
* 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 .
* query the value of the key on a parallel running instance of your Kafka Streams application .
*
*
* @param reducer a { @link Reducer } that computes a new aggregate result
* @param reducer a { @link Reducer } that computes a new aggregate result . Cannot be { @code null } .
* @param storeSupplier user defined state store supplier . Cannot be { @code null } .
* @param storeSupplier user defined state store supplier . Cannot be { @code null } .
* @return a { @link KTable } that contains "update" records with unmodified keys , and values that represent the
* @return a { @link KTable } that contains "update" records with unmodified keys , and values that represent the
* latest ( rolling ) aggregate for each key
* latest ( rolling ) aggregate for each key
* @deprectated use { @link # reduce ( Reducer , Materialized ) }
* /
* /
@Deprecated
KTable < K , V > reduce ( final Reducer < V > reducer ,
KTable < K , V > reduce ( final Reducer < V > reducer ,
final StateStoreSupplier < KeyValueStore > storeSupplier ) ;
final StateStoreSupplier < KeyValueStore > storeSupplier ) ;
/ * *
* Combine the value 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
* ( c . f . { @link # aggregate ( Initializer , Aggregator , Materialized ) } ) .
* The result is written into a local { @link KeyValueStore } ( which is basically an ever - updating materialized view )
* provided by the given { @code storeSupplier } .
* 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 ( first argument ) and the record ' s value ( second argument ) :
* < pre > { @code
* // At the example of a Reducer<Long>
* new Reducer < Long > ( ) {
* @Override
* public Long apply ( Long aggValue , Long currValue ) {
* return aggValue + currValue ;
* }
* } < / pre >
* < p >
* 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 , StateStoreSupplier ) } can be used to compute aggregate functions like sum , min , or
* max .
* < 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
* 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 { @link KeyValueStore } it must be obtained via
* { @link KafkaStreams # store ( String , QueryableStoreType ) KafkaStreams # store ( . . . ) } .
* < pre > { @code
* KafkaStreams streams = . . . // compute sum
* String queryableStoreName = "storeName" // the queryableStoreName should be the name of the store as defined by the Materialized instance
* ReadOnlyKeyValueStore < String , Long > localStore = streams . store ( queryableStoreName , QueryableStoreTypes . < String , Long > keyValueStore ( ) ) ;
* String key = "some-key" ;
* Long sumForKey = localStore . get ( key ) ; // 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 .
*
* @param reducer a { @link Reducer } that computes a new aggregate result . Cannot be { @code null } .
* @param materialized an instance of { @link Materialized } used to materialize a state store . Cannot be { @code null } .
* @return a { @link KTable } that contains "update" records with unmodified keys , and values that represent the
* latest ( rolling ) aggregate for each key
* /
KTable < K , V > reduce ( final Reducer < V > reducer ,
final Materialized < K , V , KeyValueStore < Bytes , byte [ ] > > materialized ) ;
/ * *
/ * *
* Combine the number of records in this stream by the grouped key and the defined windows .
* Combine the number of records in this stream by the grouped key and the defined windows .
* Records with { @code null } key or value are ignored .
* Records with { @code null } key or value are ignored .
@ -678,7 +767,7 @@ public interface KGroupedStream<K, V> {
* For non - local keys , a custom RPC mechanism must be implemented using { @link KafkaStreams # allMetadata ( ) } to
* 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 .
* query the value of the key on a parallel running instance of your Kafka Streams application .
*
*
* @param reducer a { @link Reducer } that computes a new aggregate result
* @param reducer a { @link Reducer } that computes a new aggregate result . Cannot be { @code null } .
* @param windows the specification of the aggregation { @link Windows }
* @param windows the specification of the aggregation { @link Windows }
* @param storeSupplier user defined state store supplier . Cannot be { @code null } .
* @param storeSupplier user defined state store supplier . Cannot be { @code null } .
* @return a windowed { @link KTable } that contains "update" records with unmodified keys , and values that represent
* @return a windowed { @link KTable } that contains "update" records with unmodified keys , and values that represent
@ -743,7 +832,7 @@ public interface KGroupedStream<K, V> {
* { @link StreamsConfig # APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG } , "queryableStoreName" is the
* { @link StreamsConfig # APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG } , "queryableStoreName" is the
* provide { @code queryableStoreName } , and "-changelog" is a fixed suffix .
* provide { @code queryableStoreName } , and "-changelog" is a fixed suffix .
* You can retrieve all generated internal topic names via { @link KafkaStreams # toString ( ) } .
* You can retrieve all generated internal topic names via { @link KafkaStreams # toString ( ) } .
* @param reducer the instance of { @link Reducer }
* @param reducer a { @link Reducer } that computes a new aggregate result . Cannot be { @code null } .
* @param sessionWindows the specification of the aggregation { @link SessionWindows }
* @param sessionWindows the specification of the aggregation { @link SessionWindows }
* @param queryableStoreName the name of the state store created from this operation ; valid characters are ASCII
* @param queryableStoreName the name of the state store created from this operation ; valid characters are ASCII
* alphanumerics , '.' , '_' and '-' . If { @code null } then this will be equivalent to { @link KGroupedStream # reduce ( Reducer , SessionWindows ) } ( ) } .
* alphanumerics , '.' , '_' and '-' . If { @code null } then this will be equivalent to { @link KGroupedStream # reduce ( Reducer , SessionWindows ) } ( ) } .
@ -778,7 +867,7 @@ public interface KGroupedStream<K, V> {
* parallel running Kafka Streams instances , and the { @link StreamsConfig configuration } parameters for
* parallel running Kafka Streams instances , and the { @link StreamsConfig configuration } parameters for
* { @link StreamsConfig # CACHE_MAX_BYTES_BUFFERING_CONFIG cache size } , and
* { @link StreamsConfig # CACHE_MAX_BYTES_BUFFERING_CONFIG cache size } , and
* { @link StreamsConfig # COMMIT_INTERVAL_MS_CONFIG commit intervall } .
* { @link StreamsConfig # COMMIT_INTERVAL_MS_CONFIG commit intervall } .
* @param reducer the instance of { @link Reducer }
* @param reducer a { @link Reducer } that computes a new aggregate result . Cannot be { @code null } .
* @param sessionWindows the specification of the aggregation { @link SessionWindows }
* @param sessionWindows the specification of the aggregation { @link SessionWindows }
* @return a windowed { @link KTable } that contains "update" records with unmodified keys , and values that represent
* @return a windowed { @link KTable } that contains "update" records with unmodified keys , and values that represent
* the latest ( rolling ) aggregate for each key within a window
* the latest ( rolling ) aggregate for each key within a window
@ -841,7 +930,7 @@ public interface KGroupedStream<K, V> {
* { @link StreamsConfig # APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG } , "queryableStoreName" is the
* { @link StreamsConfig # APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG } , "queryableStoreName" is the
* provide { @code queryableStoreName } , and "-changelog" is a fixed suffix .
* provide { @code queryableStoreName } , and "-changelog" is a fixed suffix .
* You can retrieve all generated internal topic names via { @link KafkaStreams # toString ( ) } .
* You can retrieve all generated internal topic names via { @link KafkaStreams # toString ( ) } .
* @param reducer the instance of { @link Reducer }
* @param reducer a { @link Reducer } that computes a new aggregate result . Cannot be { @code null } .
* @param sessionWindows the specification of the aggregation { @link SessionWindows }
* @param sessionWindows the specification of the aggregation { @link SessionWindows }
* @param storeSupplier user defined state store supplier . Cannot be { @code null } .
* @param storeSupplier user defined state store supplier . Cannot be { @code null } .
* @return a windowed { @link KTable } that contains "update" records with unmodified keys , and values that represent
* @return a windowed { @link KTable } that contains "update" records with unmodified keys , and values that represent
@ -905,12 +994,113 @@ public interface KGroupedStream<K, V> {
* @param < VR > the value type of the resulting { @link KTable }
* @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
* @return a { @link KTable } that contains "update" records with unmodified keys , and values that represent the
* latest ( rolling ) aggregate for each key
* latest ( rolling ) aggregate for each key
* @deprecated use { @link # aggregate ( Initializer , Aggregator , Materialized ) }
* /
* /
@Deprecated
< VR > KTable < K , VR > aggregate ( final Initializer < VR > initializer ,
< VR > KTable < K , VR > aggregate ( final Initializer < VR > initializer ,
final Aggregator < ? super K , ? super V , VR > aggregator ,
final Aggregator < ? super K , ? super V , VR > aggregator ,
final Serde < VR > aggValueSerde ,
final Serde < VR > aggValueSerde ,
final String queryableStoreName ) ;
final String queryableStoreName ) ;
/ * *
* 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 , String ) 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 provided { @code queryableStoreName } .
* Furthermore , updates to the store are sent downstream into a { @link KTable } changelog stream .
* < 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 , Serde , String ) } can be used to compute aggregate functions like
* count ( c . f . { @link # count ( String ) } ) .
* < 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
* 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 { @link KeyValueStore } it must be obtained via
* { @link KafkaStreams # store ( String , QueryableStoreType ) KafkaStreams # store ( . . . ) } :
* < pre > { @code
* KafkaStreams streams = . . . // some aggregation on value type double
* ReadOnlyKeyValueStore < String , Long > localStore = streams . store ( queryableStoreName , QueryableStoreTypes . < String , Long > keyValueStore ( ) ) ;
* String key = "some-key" ;
* Long aggForKey = localStore . get ( key ) ; // 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 .
* < p >
* For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka .
* Therefore , the store name must be a valid Kafka topic name and cannot contain characters other than ASCII
* alphanumerics , '.' , '_' and '-' .
* The changelog topic will be named "${applicationId}-${queryableStoreName}-changelog" , where "applicationId" is
* user - specified in { @link StreamsConfig } via parameter
* { @link StreamsConfig # APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG } , "queryableStoreName" is the
* provide { @code queryableStoreName } , and "-changelog" is a fixed suffix .
* You can retrieve all generated internal topic names via { @link KafkaStreams # toString ( ) } .
*
* @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 < K , VR > aggregate ( final Initializer < VR > initializer ,
final Aggregator < ? super K , ? super V , VR > aggregator ,
final Materialized < K , VR , KeyValueStore < Bytes , byte [ ] > > materialized ) ;
/ * *
* 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 , String ) 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 provided { @code queryableStoreName } .
* Furthermore , updates to the store are sent downstream into a { @link KTable } changelog stream .
* < 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 ) } can be used to compute aggregate functions like
* count ( c . f . { @link # count ( String ) } ) .
* < 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
* 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 >
* 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 { @link StreamsConfig } via parameter
* { @link StreamsConfig # APPLICATION_ID_CONFIG 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 { @link KafkaStreams # toString ( ) } .
*
* @param initializer an { @link Initializer } that computes an initial intermediate aggregation result
* @param aggregator an { @link Aggregator } that computes a new aggregate result
* @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 < 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 .
* Aggregate the values of records in this stream by the grouped key .
* Records with { @code null } key or value are ignored .
* Records with { @code null } key or value are ignored .
@ -950,7 +1140,9 @@ public interface KGroupedStream<K, V> {
* @param < VR > the value type of the resulting { @link KTable }
* @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
* @return a { @link KTable } that contains "update" records with unmodified keys , and values that represent the
* latest ( rolling ) aggregate for each key
* latest ( rolling ) aggregate for each key
* @deprecated use { @link # aggregate ( Initializer , Aggregator , Materialized ) }
* /
* /
@Deprecated
< VR > KTable < K , VR > aggregate ( final Initializer < VR > initializer ,
< VR > KTable < K , VR > aggregate ( final Initializer < VR > initializer ,
final Aggregator < ? super K , ? super V , VR > aggregator ,
final Aggregator < ? super K , ? super V , VR > aggregator ,
final Serde < VR > aggValueSerde ) ;
final Serde < VR > aggValueSerde ) ;
@ -999,7 +1191,9 @@ public interface KGroupedStream<K, V> {
* @param < VR > the value type of the resulting { @link KTable }
* @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
* @return a { @link KTable } that contains "update" records with unmodified keys , and values that represent the
* latest ( rolling ) aggregate for each key
* latest ( rolling ) aggregate for each key
* @deprecated use { @link # aggregate ( Initializer , Aggregator , Materialized ) }
* /
* /
@Deprecated
< VR > KTable < K , VR > aggregate ( final Initializer < VR > initializer ,
< VR > KTable < K , VR > aggregate ( final Initializer < VR > initializer ,
final Aggregator < ? super K , ? super V , VR > aggregator ,
final Aggregator < ? super K , ? super V , VR > aggregator ,
final StateStoreSupplier < KeyValueStore > storeSupplier ) ;
final StateStoreSupplier < KeyValueStore > storeSupplier ) ;