K - Type of keysV - Type of values@InterfaceStability.Unstable
public interface KGroupedStream<K,V>
KGroupedStream is an abstraction of a grouped record stream of KeyValue pairs.
It is an intermediate representation of a KStream in order to apply an aggregation operation on the original
KStream records.
It is an intermediate representation after a grouping of a KStream before an aggregation is applied to the
new partitions resulting in a KTable.
A KGroupedStream must be obtained from a KStream via groupByKey() or
groupBy(...).
KStream| Modifier and Type | Method and Description |
|---|---|
<T> KTable<Windowed<K>,T> |
aggregate(Initializer<T> initializer,
Aggregator<? super K,? super V,T> aggregator,
Merger<? super K,T> sessionMerger,
SessionWindows sessionWindows,
Serde<T> aggValueSerde,
StateStoreSupplier<SessionStore> storeSupplier)
Aggregate the values of records in this stream by the grouped key and defined
SessionWindows. |
<T> KTable<Windowed<K>,T> |
aggregate(Initializer<T> initializer,
Aggregator<? super K,? super V,T> aggregator,
Merger<? super K,T> sessionMerger,
SessionWindows sessionWindows,
Serde<T> aggValueSerde,
String storeName)
Aggregate the values of records in this stream by the grouped key and defined
SessionWindows. |
<VR> KTable<K,VR> |
aggregate(Initializer<VR> initializer,
Aggregator<? super K,? super V,VR> aggregator,
Serde<VR> aggValueSerde,
String storeName)
Aggregate the values of records in this stream by the grouped key.
|
<VR> KTable<K,VR> |
aggregate(Initializer<VR> initializer,
Aggregator<? super K,? super V,VR> aggregator,
StateStoreSupplier<KeyValueStore> storeSupplier)
Aggregate the values of records in this stream by the grouped key.
|
<W extends Window,VR> |
aggregate(Initializer<VR> initializer,
Aggregator<? super K,? super V,VR> aggregator,
Windows<W> windows,
Serde<VR> aggValueSerde,
String storeName)
Aggregate the values of records in this stream by the grouped key and defined windows.
|
<W extends Window,VR> |
aggregate(Initializer<VR> initializer,
Aggregator<? super K,? super V,VR> aggregator,
Windows<W> windows,
StateStoreSupplier<WindowStore> storeSupplier)
Aggregate the values of records in this stream by the grouped key and defined windows.
|
KTable<Windowed<K>,Long> |
count(SessionWindows sessionWindows,
StateStoreSupplier<SessionStore> storeSupplier)
Count the number of records in this stream by the grouped key into
SessionWindows. |
KTable<Windowed<K>,Long> |
count(SessionWindows sessionWindows,
String storeName)
Count the number of records in this stream by the grouped key into
SessionWindows. |
KTable<K,Long> |
count(StateStoreSupplier<KeyValueStore> storeSupplier)
Count the number of records in this stream by the grouped key.
|
KTable<K,Long> |
count(String storeName)
Count the number of records in this stream by the grouped key.
|
<W extends Window> |
count(Windows<W> windows,
StateStoreSupplier<WindowStore> storeSupplier)
Count the number of records in this stream by the grouped key and the defined windows.
|
<W extends Window> |
count(Windows<W> windows,
String storeName)
Count the number of records in this stream by the grouped key and the defined windows.
|
KTable<Windowed<K>,V> |
reduce(Reducer<V> reducer,
SessionWindows sessionWindows,
StateStoreSupplier<SessionStore> storeSupplier)
Combine values of this stream by the grouped key into
SessionWindows. |
KTable<Windowed<K>,V> |
reduce(Reducer<V> reducer,
SessionWindows sessionWindows,
String storeName)
Combine values of this stream by the grouped key into
SessionWindows. |
KTable<K,V> |
reduce(Reducer<V> reducer,
StateStoreSupplier<KeyValueStore> storeSupplier)
Combine the value of records in this stream by the grouped key.
|
KTable<K,V> |
reduce(Reducer<V> reducer,
String storeName)
Combine the values of records in this stream by the grouped key.
|
<W extends Window> |
reduce(Reducer<V> reducer,
Windows<W> windows,
StateStoreSupplier<WindowStore> storeSupplier)
Combine the values of records in this stream by the grouped key and the defined windows.
|
<W extends Window> |
reduce(Reducer<V> reducer,
Windows<W> windows,
String storeName)
Combine the number of records in this stream by the grouped key and the defined windows.
|
KTable<K,Long> count(String storeName)
null key or value are ignored.
The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
that can be queried using the provided storeName.
Furthermore, updates to the store are sent downstream into a KTable changelog stream.
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
To query the local KeyValueStore it must be obtained via
KafkaStreams#store(...):
KafkaStreams streams = ... // counting words
ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, 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)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "storeName" is the
provide storeName, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via KafkaStreams.toString().
KTable<K,Long> count(StateStoreSupplier<KeyValueStore> storeSupplier)
null key or value are ignored.
The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
provided by the given storeSupplier.
Furthermore, updates to the store are sent downstream into a KTable changelog stream.
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
To query the local KeyValueStore it must be obtained via
KafkaStreams#store(...).
Use StateStoreSupplier.name() to get the store name:
KafkaStreams streams = ... // counting words
String storeName = storeSupplier.name();
ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, 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)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.<W extends Window> KTable<Windowed<K>,Long> count(Windows<W> windows, String storeName)
null key or value are ignored.
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 windowed KeyValueStore (which is basically an ever-updating
materialized view) that can be queried using the provided storeName.
Windows are retained until their retention time expires (c.f. Windows.until(long)).
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.
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same window and key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
To query the local windowed KeyValueStore it must be obtained via
KafkaStreams#store(...):
KafkaStreams streams = ... // counting words
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(storeName, 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)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "storeName" is the
provide storeName, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via KafkaStreams.toString().
windows - the specification of the aggregation WindowsstoreName - the name of the underlying KTable state storeKTable that contains "update" records with unmodified keys and Long values
that represent the latest (rolling) count (i.e., number of records) for each key within a window<W extends Window> KTable<Windowed<K>,Long> count(Windows<W> windows, StateStoreSupplier<WindowStore> storeSupplier)
null key or value are ignored.
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 windowed KeyValueStore (which is basically an ever-updating
materialized view) provided by the given storeSupplier.
Windows are retained until their retention time expires (c.f. Windows.until(long)).
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.
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same window and key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
To query the local windowed KeyValueStore it must be obtained via
KafkaStreams#store(...).
Use StateStoreSupplier.name() to get the store name:
KafkaStreams streams = ... // counting words
String storeName = storeSupplier.name();
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(storeName, 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)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.windows - the specification of the aggregation WindowsstoreSupplier - user defined state store supplierKTable that contains "update" records with unmodified keys and Long values
that represent the latest (rolling) count (i.e., number of records) for each key within a windowKTable<Windowed<K>,Long> count(SessionWindows sessionWindows, String storeName)
SessionWindows.
Records with null key or value are ignored.
The result is written into a local SessionStore (which is basically an ever-updating
materialized view) that can be queried using the provided storeName.
SessionWindows are retained until their retention time expires (c.f. SessionWindows.until(long)).
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.
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same window and key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
To query the local SessionStore it must be obtained via
KafkaStreams#store(...).
Use StateStoreSupplier.name() to get the store name:
KafkaStreams streams = ... // counting words
String storeName = storeSupplier.name();
ReadOnlySessionStore<String,Long> sessionStore = streams.store(storeName, QueryableStoreTypes.<String, Long>sessionStore());
String key = "some-word";
KeyValueIterator<Windowed<String>, Long> iterator = sessionStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.sessionWindows - the specification of the aggregation SessionWindowsstoreName - the name of the state store created from this operation.KTable that contains "update" records with unmodified keys and Long values
that represent the latest (rolling) count (i.e., number of records) for each key within a windowKTable<Windowed<K>,Long> count(SessionWindows sessionWindows, StateStoreSupplier<SessionStore> storeSupplier)
SessionWindows.
Records with null key or value are ignored.
The result is written into a local SessionStore (which is basically an ever-updating materialized view)
provided by the given storeSupplier.
SessionWindows are retained until their retention time expires (c.f. SessionWindows.until(long)).
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.
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same window and key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
To query the local SessionStore it must be obtained via
KafkaStreams#store(...).
Use StateStoreSupplier.name() to get the store name:
KafkaStreams streams = ... // counting words
String storeName = storeSupplier.name();
ReadOnlySessionStore<String,Long> sessionStore = streams.store(storeName, QueryableStoreTypes.<String, Long>sessionStore());
String key = "some-word";
KeyValueIterator<Windowed<String>, Long> iterator = sessionStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.sessionWindows - the specification of the aggregation SessionWindowsstoreSupplier - user defined state store supplierKTable that contains "update" records with unmodified keys and Long values
that represent the latest (rolling) count (i.e., number of records) for each key within a windowKTable<K,V> reduce(Reducer<V> reducer, String storeName)
null key or value are ignored.
Combining implies that the type of the aggregate result is the same as the type of the input value
(c.f. aggregate(Initializer, Aggregator, Serde, String)).
The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
that can be queried using the provided storeName.
Furthermore, updates to the store are sent downstream into a KTable changelog stream.
The specified 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 Reducer is 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 configuration parameters for
cache size, and
commit intervall.
To query the local KeyValueStore it must be obtained via
KafkaStreams#store(...):
KafkaStreams streams = ... // compute sum
ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, 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)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "storeName" is the
provide storeName, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via KafkaStreams.toString().
KTable<K,V> reduce(Reducer<V> reducer, StateStoreSupplier<KeyValueStore> storeSupplier)
null key or value are ignored.
Combining implies that the type of the aggregate result is the same as the type of the input value
(c.f. aggregate(Initializer, Aggregator, StateStoreSupplier)).
The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
provided by the given storeSupplier.
Furthermore, updates to the store are sent downstream into a KTable changelog stream.
The specified 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 Reducer is not applied and the new aggregate will be the record's
value as-is.
Thus, reduce(Reducer, StateStoreSupplier) 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 configuration parameters for
cache size, and
commit intervall.
To query the local KeyValueStore it must be obtained via
KafkaStreams#store(...).
Use StateStoreSupplier.name() to get the store name:
KafkaStreams streams = ... // compute sum
String storeName = storeSupplier.name();
ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, 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)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.<W extends Window> KTable<Windowed<K>,V> reduce(Reducer<V> reducer, Windows<W> windows, String storeName)
null key or value are ignored.
Combining implies that the type of the aggregate result is the same as the type of the input value
(c.f. aggregate(Initializer, Aggregator, Windows, Serde, String)).
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 windowed KeyValueStore (which is basically an ever-updating
materialized view) that can be queried using the provided storeName.
Windows are retained until their retention time expires (c.f. Windows.until(long)).
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.
The specified 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 Reducer is not applied and the new aggregate will be the record's
value as-is.
Thus, reduce(Reducer, Windows, 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 window and key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
To query the local windowed KeyValueStore it must be obtained via
KafkaStreams#store(...):
KafkaStreams streams = ... // compute sum
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(storeName, QueryableStoreTypes.<String, Long>windowStore());
String key = "some-key";
long fromTime = ...;
long toTime = ...;
WindowStoreIterator<Long> sumForKeyForWindows = localWindowStore.fetch(key, timeFrom, timeTo); // key must be local (application state is shared over all running Kafka Streams instances)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "storeName" is the
provide storeName, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via KafkaStreams.toString().
reducer - a Reducer that computes a new aggregate resultwindows - the specification of the aggregation WindowsstoreName - the name of the state store created from this operationKTable that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window<W extends Window> KTable<Windowed<K>,V> reduce(Reducer<V> reducer, Windows<W> windows, StateStoreSupplier<WindowStore> storeSupplier)
null key or value are ignored.
Combining implies that the type of the aggregate result is the same as the type of the input value
(c.f. aggregate(Initializer, Aggregator, Windows, Serde, String)).
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 windowed KeyValueStore (which is basically an ever-updating
materialized view) provided by the given storeSupplier.
Windows are retained until their retention time expires (c.f. Windows.until(long)).
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.
The specified 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 Reducer is not applied and the new aggregate will be the record's
value as-is.
Thus, reduce(Reducer, Windows, StateStoreSupplier) can be used to compute aggregate functions like sum,
min, or max.
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same window and key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
To query the local windowed KeyValueStore it must be obtained via
KafkaStreams#store(...).
Use StateStoreSupplier.name() to get the store name:
KafkaStreams streams = ... // compute sum
Sting storeName = storeSupplier.name();
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(storeName, QueryableStoreTypes.<String, Long>windowStore());
String key = "some-key";
long fromTime = ...;
long toTime = ...;
WindowStoreIterator<Long> sumForKeyForWindows = localWindowStore.fetch(key, timeFrom, timeTo); // key must be local (application state is shared over all running Kafka Streams instances)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.reducer - a Reducer that computes a new aggregate resultwindows - the specification of the aggregation WindowsstoreSupplier - user defined state store supplierKTable that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a windowKTable<Windowed<K>,V> reduce(Reducer<V> reducer, SessionWindows sessionWindows, String storeName)
SessionWindows.
Records with null key or value are ignored.
Combining implies that the type of the aggregate result is the same as the type of the input value
(c.f. aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde, String)).
The result is written into a local SessionStore (which is basically an ever-updating
materialized view) that can be queried using the provided storeName.
SessionWindows are retained until their retention time expires (c.f. SessionWindows.until(long)).
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.
The specified 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 Reducer is not applied and the new aggregate will be the record's
value as-is.
Thus, reduce(Reducer, SessionWindows, 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 window and key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
To query the local SessionStore it must be obtained via
KafkaStreams#store(...):
KafkaStreams streams = ... // compute sum
ReadOnlySessionStore<String,Long> sessionStore = streams.store(storeName, QueryableStoreTypes.<String, Long>sessionStore());
String key = "some-key";
KeyValueIterator<Windowed<String>, Long> sumForKeyForSession = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "storeName" is the
provide storeName, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via KafkaStreams.toString().
reducer - the instance of ReducersessionWindows - the specification of the aggregation SessionWindowsstoreName - the name of the state store created from this operationKTable that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a windowKTable<Windowed<K>,V> reduce(Reducer<V> reducer, SessionWindows sessionWindows, StateStoreSupplier<SessionStore> storeSupplier)
SessionWindows.
Records with null key or value are ignored.
Combining implies that the type of the aggregate result is the same as the type of the input value
(c.f. aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde, String)).
The result is written into a local SessionStore (which is basically an ever-updating materialized view)
provided by the given storeSupplier.
SessionWindows are retained until their retention time expires (c.f. SessionWindows.until(long)).
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.
The specified 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 Reducer is not applied and the new aggregate will be the record's
value as-is.
Thus, reduce(Reducer, SessionWindows, StateStoreSupplier) can be used to compute aggregate functions like
sum, min, or max.
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same window and key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
To query the local SessionStore it must be obtained via
KafkaStreams#store(...).
Use StateStoreSupplier.name() to get the store name:
KafkaStreams streams = ... // compute sum
Sting storeName = storeSupplier.name();
ReadOnlySessionStore<String,Long> sessionStore = streams.store(storeName, QueryableStoreTypes.<String, Long>sessionStore());
String key = "some-key";
KeyValueIterator<Windowed<String>, Long> sumForKeyForSession = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "storeName" is the
provide storeName, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via KafkaStreams.toString().
reducer - the instance of ReducersessionWindows - the specification of the aggregation SessionWindowsstoreSupplier - user defined state store supplierKTable that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window<VR> KTable<K,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Serde<VR> aggValueSerde, String storeName)
null key or value are ignored.
Aggregating is a generalization of combining via reduce(...) as it, for example,
allows the result to have a different type than the input values.
The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
that can be queried using the provided storeName.
Furthermore, updates to the store are sent downstream into a KTable changelog stream.
The specified 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 Aggregator is applied for each input record and computes a new aggregate using the current
aggregate (or for the very first record using the intermediate aggregation result provided via the
Initializer) and the record's value.
Thus, aggregate(Initializer, Aggregator, Serde, String) can be used to compute aggregate functions like
count (c.f. count(String)).
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 configuration parameters for
cache size, and
commit intervall.
To query the local KeyValueStore it must be obtained via
KafkaStreams#store(...):
KafkaStreams streams = ... // some aggregation on value type double
ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, 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)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "storeName" is the
provide storeName, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via KafkaStreams.toString().
VR - the value type of the resulting KTableinitializer - an Initializer that computes an initial intermediate aggregation resultaggregator - an Aggregator that computes a new aggregate resultaggValueSerde - aggregate value serdes for materializing the aggregated table,
if not specified the default serdes defined in the configs will be usedstoreName - the name of the state store created from this operationKTable that contains "update" records with unmodified keys, and values that represent the
latest (rolling) aggregate for each key<VR> KTable<K,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, StateStoreSupplier<KeyValueStore> storeSupplier)
null key or value are ignored.
Aggregating is a generalization of combining via reduce(...) as it,
for example, allows the result to have a different type than the input values.
The result is written into a local KeyValueStore (which is basically an ever-updating materialized view)
provided by the given storeSupplier.
Furthermore, updates to the store are sent downstream into a KTable changelog stream.
The specified 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 Aggregator is applied for each input record and computes a new aggregate using the current
aggregate (or for the very first record using the intermediate aggregation result provided via the
Initializer) and the record's value.
Thus, aggregate(Initializer, Aggregator, StateStoreSupplier) can be used to compute aggregate functions
like count (c.f. count(String)).
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 configuration parameters for
cache size, and
commit intervall.
To query the local KeyValueStore it must be obtained via
KafkaStreams#store(...).
Use StateStoreSupplier.name() to get the store name:
KafkaStreams streams = ... // some aggregation on value type double
Sting storeName = storeSupplier.name();
ReadOnlyKeyValueStore<String,Long> localStore = streams.store(storeName, 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)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.VR - the value type of the resulting KTableinitializer - an Initializer that computes an initial intermediate aggregation resultaggregator - an Aggregator that computes a new aggregate resultstoreSupplier - user defined state store supplierKTable that contains "update" records with unmodified keys, and values that represent the
latest (rolling) aggregate for each key<W extends Window,VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Windows<W> windows, Serde<VR> aggValueSerde, String storeName)
null key or value are ignored.
Aggregating is a generalization of combining via reduce(...) as it,
for example, allows the result to have a different type than the input values.
The 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 windowed KeyValueStore (which is basically an ever-updating
materialized view) that can be queried using the provided storeName.
Windows are retained until their retention time expires (c.f. Windows.until(long)).
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.
The specified Initializer is applied once per window 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 Aggregator is applied for each input record and computes a new aggregate using the current
aggregate (or for the very first record using the intermediate aggregation result provided via the
Initializer) and the record's value.
Thus, aggregate(Initializer, Aggregator, Windows, Serde, String) can be used to compute aggregate
functions like count (c.f. count(String)).
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same window and key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
To query the local windowed KeyValueStore it must be obtained via
KafkaStreams#store(...):
KafkaStreams streams = ... // some windowed aggregation on value type double
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(storeName, QueryableStoreTypes.<String, Long>windowStore());
String key = "some-key";
long fromTime = ...;
long toTime = ...;
WindowStoreIterator<Long> aggForKeyForWindows = localWindowStore.fetch(key, timeFrom, timeTo); // key must be local (application state is shared over all running Kafka Streams instances)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.
For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka.
The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is
user-specified in StreamsConfig via parameter
APPLICATION_ID_CONFIG, "storeName" is the
provide storeName, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via KafkaStreams.toString().
VR - the value type of the resulting KTableinitializer - an Initializer that computes an initial intermediate aggregation resultaggregator - an Aggregator that computes a new aggregate resultwindows - the specification of the aggregation WindowsaggValueSerde - aggregate value serdes for materializing the aggregated table,
if not specified the default serdes defined in the configs will be usedstoreName - the name of the state store created from this operationKTable that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window<W extends Window,VR> KTable<Windowed<K>,VR> aggregate(Initializer<VR> initializer, Aggregator<? super K,? super V,VR> aggregator, Windows<W> windows, StateStoreSupplier<WindowStore> storeSupplier)
null key or value are ignored.
Aggregating is a generalization of combining via
reduce(...) as it, for example, allows the result to have a different type than the input values.
The 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 windowed KeyValueStore (which is basically an ever-updating
materialized view) provided by the given storeSupplier.
Windows are retained until their retention time expires (c.f. Windows.until(long)).
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.
The specified Initializer is applied once per window 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 Aggregator is applied for each input record and computes a new aggregate using the current
aggregate (or for the very first record using the intermediate aggregation result provided via the
Initializer) and the record's value.
Thus, aggregate(Initializer, Aggregator, Windows, StateStoreSupplier) can be used to compute aggregate
functions like count (c.f. count(String)).
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same window and key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
To query the local windowed KeyValueStore it must be obtained via
KafkaStreams#store(...).
Use StateStoreSupplier.name() to get the store name:
KafkaStreams streams = ... // some windowed aggregation on value type Long
Sting storeName = storeSupplier.name();
ReadOnlyWindowStore<String,Long> localWindowStore = streams.store(storeName, QueryableStoreTypes.<String, Long>windowStore());
String key = "some-key";
long fromTime = ...;
long toTime = ...;
WindowStoreIterator<Long> aggForKeyForWindows = localWindowStore.fetch(key, timeFrom, timeTo); // key must be local (application state is shared over all running Kafka Streams instances)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.VR - the value type of the resulting KTableinitializer - an Initializer that computes an initial intermediate aggregation resultaggregator - an Aggregator that computes a new aggregate resultwindows - the specification of the aggregation WindowsstoreSupplier - user defined state store supplierKTable that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window<T> KTable<Windowed<K>,T> aggregate(Initializer<T> initializer, Aggregator<? super K,? super V,T> aggregator, Merger<? super K,T> sessionMerger, SessionWindows sessionWindows, Serde<T> aggValueSerde, String storeName)
SessionWindows.
Records with null key or value are ignored.
Aggregating is a generalization of combining via
reduce(...) as it, for example, allows the result to have a different type than the input values.
The result is written into a local SessionStore (which is basically an ever-updating
materialized view) that can be queried using the provided storeName.
SessionWindows are retained until their retention time expires (c.f. SessionWindows.until(long)).
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.
The specified Initializer is applied once per session 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 Aggregator is applied for each input record and computes a new aggregate using the current
aggregate (or for the very first record using the intermediate aggregation result provided via the
Initializer) and the record's value.
Thus, aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde, String) can be used to compute
aggregate functions like count (c.f. count(String))
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same window and key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
To query the local SessionStore it must be obtained via
KafkaStreams#store(...).
Use StateStoreSupplier.name() to get the store name:
KafkaStreams streams = ... // some windowed aggregation on value type double
Sting storeName = storeSupplier.name();
ReadOnlySessionStore<String, Long> sessionStore = streams.store(storeName, QueryableStoreTypes.<String, Long>sessionStore());
String key = "some-key";
KeyValueIterator<Windowed<String>, Long> aggForKeyForSession = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.T - the value type of the resulting KTableinitializer - the instance of Initializeraggregator - the instance of AggregatorsessionMerger - the instance of MergersessionWindows - the specification of the aggregation SessionWindowsaggValueSerde - aggregate value serdes for materializing the aggregated table,
if not specified the default serdes defined in the configs will be usedstoreName - the name of the state store created from this operationKTable that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window<T> KTable<Windowed<K>,T> aggregate(Initializer<T> initializer, Aggregator<? super K,? super V,T> aggregator, Merger<? super K,T> sessionMerger, SessionWindows sessionWindows, Serde<T> aggValueSerde, StateStoreSupplier<SessionStore> storeSupplier)
SessionWindows.
Records with null key or value are ignored.
Aggregating is a generalization of combining via
reduce(...) as it, for example, allows the result to have a different type than the input values.
The result is written into a local SessionStore (which is basically an ever-updating materialized view)
provided by the given storeSupplier.
SessionWindows are retained until their retention time expires (c.f. SessionWindows.until(long)).
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.
The specified Initializer is applied once per session 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 Aggregator is applied for each input record and computes a new aggregate using the current
aggregate (or for the very first record using the intermediate aggregation result provided via the
Initializer) and the record's value.
Thus, #aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde, StateStoreSupplier) can be used
to compute aggregate functions like count (c.f. count(String)).
Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
the same window and key.
The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
parallel running Kafka Streams instances, and the configuration parameters for
cache size, and
commit intervall.
To query the local SessionStore it must be obtained via
KafkaStreams#store(...).
Use StateStoreSupplier.name() to get the store name:
KafkaStreams streams = ... // some windowed aggregation on value type double
Sting storeName = storeSupplier.name();
ReadOnlySessionStore<String, Long> sessionStore = streams.store(storeName, QueryableStoreTypes.<String, Long>sessionStore());
String key = "some-key";
KeyValueIterator<Windowed<String>, Long> aggForKeyForSession = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to
query the value of the key on a parallel running instance of your Kafka Streams application.T - the value type of the resulting KTableinitializer - the instance of Initializeraggregator - the instance of AggregatorsessionMerger - the instance of MergersessionWindows - the specification of the aggregation SessionWindowsaggValueSerde - aggregate value serdes for materializing the aggregated table,
if not specified the default serdes defined in the configs will be usedstoreSupplier - user defined state store supplierKTable that contains "update" records with unmodified keys, and values that represent
the latest (rolling) aggregate for each key within a window