Interface TimeWindowedCogroupedKStream<K,V>
- Type Parameters:
K
- Type of keysV
- Type of values
TimeWindowedCogroupKStream
is an abstraction of a windowed record stream of KeyValue
pairs.
It is an intermediate representation of a CogroupedKStream
in order to apply a windowed aggregation operation
on the original KGroupedStream
records resulting in a windowed KTable
(a KTable
is a KTable
with key type Windowed
).
The specified windows
define either hopping time windows that can be overlapping or tumbling (c.f.
TimeWindows
) or they define landmark windows (c.f. UnlimitedWindows
).
The result is written into a local WindowStore
(which is basically an ever-updating
materialized view) that can be queried using the name provided in the Materialized
instance.
Furthermore, updates to the store are sent downstream into a windowed KTable
changelog stream, where
"windowed" implies that the KTable
key is a combined key of the original record key and a window ID.
New events are added to windows until their grace period ends (see TimeWindows.grace(Duration)
).
A TimeWindowedCogroupedKStream
must be obtained from a CogroupedKStream
via
CogroupedKStream.windowedBy(Windows)
.
- See Also:
-
Method Summary
Modifier and TypeMethodDescriptionaggregate
(Initializer<V> initializer) Aggregate the values of records in this stream by the grouped key and defined windows.aggregate
(Initializer<V> initializer, Materialized<K, V, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in this stream by the grouped key and defined windows.aggregate
(Initializer<V> initializer, Named named) Aggregate the values of records in this stream by the grouped key and defined windows.aggregate
(Initializer<V> initializer, Named named, Materialized<K, V, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in this stream by the grouped key and defined windows.
-
Method Details
-
aggregate
Aggregate the values of records in this stream by the grouped key and defined windows. Records withnull
key or value are ignored. The result is written into a localWindowStore
(which is basically an ever-updating materialized view). Furthermore, updates to the store are sent downstream into aKTable
changelog stream.The specified
Initializer
is applied directly before the first input record (per key) in each window is processed to provide an initial intermediate aggregation result that is used to process the first record for the window (per key). The specifiedAggregator
(as specified inKGroupedStream.cogroup(Aggregator)
orCogroupedKStream.cogroup(KGroupedStream, 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 theInitializer
) and the record's value. Thus,aggregate()
can be used to compute aggregate functions like count or sum etc.The default key and value serde from the config will be used for serializing the result. If a different serde is required then you should use
aggregate(Initializer, Materialized)
. Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to the same window and key. The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of parallel running Kafka Streams instances, and theconfiguration
parameters forcache size
, andcommit interval
.For failure and recovery the store (which always will be of type
TimestampedWindowStore
) will be backed by an internal changelog topic that will be created in Kafka. The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is user-specified inStreamsConfig
via parameterAPPLICATION_ID_CONFIG
, "internalStoreName" is an internal name and "-changelog" is a fixed suffix. Note that the internal store name may not be queryable through Interactive Queries.You can retrieve all generated internal topic names via
Topology.describe()
.- Parameters:
initializer
- anInitializer
that computes an initial intermediate aggregation result. Cannot benull
.- Returns:
- a windowed
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key within a window
-
aggregate
Aggregate the values of records in this stream by the grouped key and defined windows. Records withnull
key or value are ignored. The result is written into a localWindowStore
(which is basically an ever-updating materialized view). Furthermore, updates to the store are sent downstream into aKTable
changelog stream.The specified
Initializer
is applied directly before the first input record (per key) in each window is processed to provide an initial intermediate aggregation result that is used to process the first record for the window (per key). The specifiedAggregator
(as specified inKGroupedStream.cogroup(Aggregator)
orCogroupedKStream.cogroup(KGroupedStream, 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 theInitializer
) and the record's value. Thus,aggregate()
can be used to compute aggregate functions like count or sum etc.The default key and value serde from the config will be used for serializing the result. If a different serde is required then you should use
aggregate(Initializer, Named, Materialized)
. Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to the same window and key. The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of parallel running Kafka Streams instances, and theconfiguration
parameters forcache size
, andcommit interval
.For failure and recovery the store (which always will be of type
TimestampedWindowStore
) will be backed by an internal changelog topic that will be created in Kafka. The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is user-specified inStreamsConfig
via parameterAPPLICATION_ID_CONFIG
, "internalStoreName" is an internal name and "-changelog" is a fixed suffix. Note that the internal store name may not be queryable through Interactive Queries.You can retrieve all generated internal topic names via
Topology.describe()
.- Parameters:
initializer
- anInitializer
that computes an initial intermediate aggregation result. Cannot benull
.named
- aNamed
config used to name the processor in the topology. Cannot benull
.- Returns:
- a windowed
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key within a window
-
aggregate
KTable<Windowed<K>,V> aggregate(Initializer<V> initializer, Materialized<K, V, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in this stream by the grouped key and defined windows. Records withnull
key or value are ignored. The result is written into a localWindowStore
(which is basically an ever-updating materialized view) that can be queried using the store name as provided withMaterialized
. Furthermore, updates to the store are sent downstream into aKTable
changelog stream.The specified
Initializer
is applied directly before the first input record (per key) in each window is processed to provide an initial intermediate aggregation result that is used to process the first record for the window (per key). The specifiedAggregator
(as specified inKGroupedStream.cogroup(Aggregator)
orCogroupedKStream.cogroup(KGroupedStream, 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 theInitializer
) and the record's value. Thus,aggregate()
can be used to compute aggregate functions like count or sum etc.Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to the same window and key if caching is enabled on the
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 theconfiguration
parameters forcache size
, andcommit interval
.To query the local
ReadOnlyWindowStore
it 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<K, ValueAndTimestamp<V>> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<V>>timestampedWindowStore()); K key = "some-word"; long fromTime = ...; long toTime = ...; WindowStoreIterator<ValueAndTimestamp<V>> aggregateStore = localWindowStore.fetch(key, timeFrom, timeTo); // key must be local (application state is shared over all running Kafka Streams instances)
KafkaStreams.metadataForAllStreamsClients()
to query the value of the key on a parallel running instance of your Kafka Streams application.For failure and recovery the store (which always will be of type
TimestampedWindowStore
-- regardless of what is specified in the parametermaterialized
) will be backed by an internal changelog topic that will be created in Kafka. Therefore, the store name defined by theMaterialized
instance must be a valid Kafka topic name and cannot contain characters other than ASCII alphanumerics, '.', '_' and '-'. The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is user-specified inStreamsConfig
via parameterAPPLICATION_ID_CONFIG
, "storeName" is the provide store name defined inMaterialized
, and "-changelog" is a fixed suffix.You can retrieve all generated internal topic names via
Topology.describe()
.- Parameters:
initializer
- anInitializer
that computes an initial intermediate aggregation result. Cannot benull
.materialized
- aMaterialized
config used to materialize a state store. Cannot benull
.- Returns:
- a windowed
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key within a window
-
aggregate
KTable<Windowed<K>,V> aggregate(Initializer<V> initializer, Named named, Materialized<K, V, WindowStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in this stream by the grouped key and defined windows. Records withnull
key or value are ignored. The result is written into a localWindowStore
(which is basically an ever-updating materialized view) that can be queried using the store name as provided withMaterialized
. Furthermore, updates to the store are sent downstream into aKTable
changelog stream.The specified
Initializer
is applied directly before the first input record (per key) in each window is processed to provide an initial intermediate aggregation result that is used to process the first record for the window (per key). The specifiedAggregator
(as specified inKGroupedStream.cogroup(Aggregator)
orCogroupedKStream.cogroup(KGroupedStream, 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 theInitializer
) and the record's value. Thus,aggregate()
can be used to compute aggregate functions like count or sum etc.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
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 theconfiguration
parameters forcache size
, andcommit interval
.To query the local
ReadOnlyWindowStore
it 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<K, ValueAndTimestamp<V>> localWindowStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<V>>timestampedWindowStore()); K key = "some-word"; long fromTime = ...; long toTime = ...; WindowStoreIterator<ValueAndTimestamp<V>> aggregateStore = localWindowStore.fetch(key, timeFrom, timeTo); // key must be local (application state is shared over all running Kafka Streams instances)
KafkaStreams.metadataForAllStreamsClients()
to query the value of the key on a parallel running instance of your Kafka Streams application.For failure and recovery the store (which always will be of type
TimestampedWindowStore
-- regardless of what is specified in the parametermaterialized
) will be backed by an internal changelog topic that will be created in Kafka. Therefore, the store name defined by theMaterialized
instance must be a valid Kafka topic name and cannot contain characters other than ASCII alphanumerics, '.', '_' and '-'. The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is user-specified inStreamsConfig
via parameterAPPLICATION_ID_CONFIG
, "storeName" is the provide store name defined inMaterialized
, and "-changelog" is a fixed suffix.You can retrieve all generated internal topic names via
Topology.describe()
.- Parameters:
initializer
- anInitializer
that computes an initial intermediate aggregation result. Cannot benull
.named
- aNamed
config used to name the processor in the topology. Cannot benull
.materialized
- aMaterialized
config used to materialize a state store. Cannot benull
.- Returns:
- a windowed
KTable
that contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key within a window
-