K
- Type of keysVOut
- Type of values after aggpublic interface CogroupedKStream<K,VOut>
CogroupedKStream
is an abstraction of multiple grouped record streams of KeyValue
pairs.
It is an intermediate representation after a grouping of KStream
s, before the
aggregations are applied to the new partitions resulting in a KTable
.
A CogroupedKStream
must be obtained from a KGroupedStream
via
cogroup(...)
.
Modifier and Type | Method and Description |
---|---|
KTable<K,VOut> |
aggregate(Initializer<VOut> initializer)
Aggregate the values of records in these streams by the grouped key.
|
KTable<K,VOut> |
aggregate(Initializer<VOut> initializer,
Materialized<K,VOut,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Aggregate the values of records in these streams by the grouped key.
|
KTable<K,VOut> |
aggregate(Initializer<VOut> initializer,
Named named)
Aggregate the values of records in these streams by the grouped key.
|
KTable<K,VOut> |
aggregate(Initializer<VOut> initializer,
Named named,
Materialized<K,VOut,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Aggregate the values of records in these streams by the grouped key.
|
<VIn> CogroupedKStream<K,VOut> |
cogroup(KGroupedStream<K,VIn> groupedStream,
Aggregator<? super K,? super VIn,VOut> aggregator)
Add an already
grouped KStream to this CogroupedKStream . |
SessionWindowedCogroupedKStream<K,VOut> |
windowedBy(SessionWindows windows)
Create a new
SessionWindowedCogroupedKStream instance that can be used to perform session
windowed aggregations. |
TimeWindowedCogroupedKStream<K,VOut> |
windowedBy(SlidingWindows windows)
Create a new
TimeWindowedCogroupedKStream instance that can be used to perform sliding
windowed aggregations. |
<W extends Window> |
windowedBy(Windows<W> windows)
Create a new
TimeWindowedCogroupedKStream instance that can be used to perform windowed
aggregations. |
<VIn> CogroupedKStream<K,VOut> cogroup(KGroupedStream<K,VIn> groupedStream, Aggregator<? super K,? super VIn,VOut> aggregator)
grouped KStream
to this CogroupedKStream
.
The added grouped KStream
must have the same number of partitions as all existing
streams of this CogroupedKStream
.
If this is not the case, you would need to call KStream.repartition(Repartitioned)
before
grouping
the KStream
and specify the "correct" number of
partitions via Repartitioned
parameter.
The specified Aggregator
is applied in the actual aggregation
step for
each input record and computes a new aggregate using the current aggregate (or for the very first record per key
using the initial intermediate aggregation result provided via the Initializer
that is passed into
aggregate(Initializer)
) and the record's value.
VIn
- Type of input valuesgroupedStream
- a group streamaggregator
- an Aggregator
that computes a new aggregate resultCogroupedKStream
KTable<K,VOut> aggregate(Initializer<VOut> initializer)
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 by the given store name in materialized
.
Furthermore, updates to the store are sent downstream into a KTable
changelog stream.
To compute the aggregation the corresponding Aggregator
as specified in
cogroup(...)
is used per input stream.
The specified Initializer
is applied once per key, directly before the first input record per key is
processed to provide an initial intermediate aggregation result that is used to process the first record.
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 interval
.
To query the local ReadOnlyKeyValueStore
it must be obtained via
KafkaStreams#store(...)
:
KafkaStreams streams = ... // some aggregation on value type double
String queryableStoreName = "storeName" // the store name should be the name of the store as defined by the Materialized instance
ReadOnlyKeyValueStore<K, ValueAndTimestamp<VOut>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<VOut>> timestampedKeyValueStore());
K key = "some-key";
ValueAndTimestamp<VOut> 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.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 TimestampedKeyValueStore
) will be backed by
an internal changelog topic that will be created in Kafka.
Therefore, the store name defined by the Materialized 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 in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "storeName" is a generated value, and
"-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via Topology.describe()
.
initializer
- an Initializer
that computes an initial intermediate aggregation
result. Cannot be null
.KTable
that contains "update" records with unmodified keys, and values that
represent the latest (rolling) aggregate for each keyKTable<K,VOut> aggregate(Initializer<VOut> initializer, Named named)
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 by the given store name in materialized
.
Furthermore, updates to the store are sent downstream into a KTable
changelog stream.
To compute the aggregation the corresponding Aggregator
as specified in
cogroup(...)
is used per input stream.
The specified Initializer
is applied once per key, directly before the first input record per key is
processed to provide an initial intermediate aggregation result that is used to process the first record.
The specified Named
is applied once to the processor combining the grouped streams.
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 interval
.
To query the local ReadOnlyKeyValueStore
it must be obtained via
KafkaStreams#store(...)
:
KafkaStreams streams = ... // some aggregation on value type double
String queryableStoreName = "storeName" // the store name should be the name of the store as defined by the Materialized instance
ReadOnlyKeyValueStore<K, ValueAndTimestamp<VOut>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<VOut>> timestampedKeyValueStore());
K key = "some-key";
ValueAndTimestamp<VOut> 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.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 TimestampedKeyValueStore
) will be backed by
an internal changelog topic that will be created in Kafka.
Therefore, the store name defined by the Materialized 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 in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "storeName" is the provide store name defined
in Materialized
, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via Topology.describe()
.
initializer
- an Initializer
that computes an initial intermediate aggregation
result. Cannot be null
.named
- name the processor. Cannot be null
.KTable
that contains "update" records with unmodified keys, and values that
represent the latest (rolling) aggregate for each keyKTable<K,VOut> aggregate(Initializer<VOut> initializer, Materialized<K,VOut,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
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 by the given store name in materialized
.
Furthermore, updates to the store are sent downstream into a KTable
changelog stream.
To compute the aggregation the corresponding Aggregator
as specified in
cogroup(...)
is used per input stream.
The specified Initializer
is applied once per key, directly before the first input record per key is
processed to provide an initial intermediate aggregation result that is used to process the first record.
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 interval
.
To query the local ReadOnlyKeyValueStore
it must be obtained via
KafkaStreams#store(...)
:
KafkaStreams streams = ... // some aggregation on value type double
String queryableStoreName = "storeName" // the store name should be the name of the store as defined by the Materialized instance
ReadOnlyKeyValueStore<K, ValueAndTimestamp<VOut>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<VOut>> timestampedKeyValueStore());
K key = "some-key";
ValueAndTimestamp<VOut> 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.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 TimestampedKeyValueStore
-- regardless of what
is specified in the parameter materialized
) will be backed by an internal changelog topic that will be created in Kafka.
Therefore, the store name defined by the Materialized 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 in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "storeName" is the provide store name defined
in Materialized
, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via Topology.describe()
.
initializer
- an Initializer
that computes an initial intermediate aggregation
result. Cannot be null
.materialized
- an instance of Materialized
used to materialize a state store.
Cannot be null
.KTable
that contains "update" records with unmodified keys, and values that
represent the latest (rolling) aggregate for each keyKTable<K,VOut> aggregate(Initializer<VOut> initializer, Named named, Materialized<K,VOut,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
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 by the given store name in materialized
.
Furthermore, updates to the store are sent downstream into a KTable
changelog stream.
To compute the aggregation the corresponding Aggregator
as specified in
cogroup(...)
is used per input stream.
The specified Initializer
is applied once per key, directly before the first input record per key is
processed to provide an initial intermediate aggregation result that is used to process the first record.
The specified Named
is used to name the processor combining the grouped streams.
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 interval
.
To query the local ReadOnlyKeyValueStore
it must be obtained via
KafkaStreams#store(...)
:
KafkaStreams streams = ... // some aggregation on value type double
String queryableStoreName = "storeName" // the store name should be the name of the store as defined by the Materialized instance
ReadOnlyKeyValueStore<K, ValueAndTimestamp<VOut>> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, ValueAndTimestamp<VOut>> timestampedKeyValueStore());
K key = "some-key";
ValueAndTimestamp<VOut> 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.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 TimestampedKeyValueStore
-- regardless of what
is specified in the parameter materialized
) will be backed by an internal changelog topic that will be created in Kafka.
Therefore, the store name defined by the Materialized 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 in StreamsConfig
via parameter
APPLICATION_ID_CONFIG
, "storeName" is the provide store name defined
in Materialized
, and "-changelog" is a fixed suffix.
You can retrieve all generated internal topic names via Topology.describe()
.
initializer
- an Initializer
that computes an initial intermediate aggregation
result. Cannot be null
.materialized
- an instance of Materialized
used to materialize a state store.
Cannot be null
.named
- name the processors. Cannot be null
.KTable
that contains "update" records with unmodified keys, and values that
represent the latest (rolling) aggregate for each key<W extends Window> TimeWindowedCogroupedKStream<K,VOut> windowedBy(Windows<W> windows)
TimeWindowedCogroupedKStream
instance that can be used to perform windowed
aggregations.W
- the window typewindows
- the specification of the aggregation Windows
TimeWindowedCogroupedKStream
TimeWindowedCogroupedKStream<K,VOut> windowedBy(SlidingWindows windows)
TimeWindowedCogroupedKStream
instance that can be used to perform sliding
windowed aggregations.windows
- the specification of the aggregation SlidingWindows
TimeWindowedCogroupedKStream
SessionWindowedCogroupedKStream<K,VOut> windowedBy(SessionWindows windows)
SessionWindowedCogroupedKStream
instance that can be used to perform session
windowed aggregations.windows
- the specification of the aggregation SessionWindows
SessionWindowedCogroupedKStream