Interface SessionWindowedCogroupedKStream<K,V> 
- Type Parameters:
- K- Type of keys
- V- Type of values
SessionWindowedCogroupKStream 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).
 
 SessionWindows are dynamic data driven windows.
 They have no fixed time boundaries, rather the size of the window is determined by the records.
 
 The result is written into a local SessionStore (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 sessions until their grace period ends (see SessionWindows.ofInactivityGapAndGrace(Duration, Duration)).
 
 A SessionWindowedCogroupedKStream must be obtained from a CogroupedKStream via
 CogroupedKStream.windowedBy(SessionWindows).
- See Also:
- 
Method SummaryModifier and TypeMethodDescriptionAggregate the values of records in these streams by the grouped key and defined sessions.aggregate(Initializer<V> initializer, Merger<? super K, V> sessionMerger, Materialized<K, V, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in these streams by the grouped key and defined sessions.Aggregate the values of records in these streams by the grouped key and defined sessions.aggregate(Initializer<V> initializer, Merger<? super K, V> sessionMerger, Named named, Materialized<K, V, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in these streams by the grouped key and defined sessions.
- 
Method Details- 
aggregateAggregate the values of records in these streams by the grouped key and defined sessions. Note that sessions are generated on a per-key basis and records with different keys create independent sessions. Records withnullkey or value are ignored. The result is written into a localSessionStore(which is basically an ever-updating materialized view). Furthermore, updates to the store are sent downstream into aKTablechangelog stream.The specified Initializeris applied directly before the first input record per session is processed to provide an initial intermediate aggregation result that is used to process the first record per session. 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. The specifiedMergeris used to merge two existing sessions into one, i.e., when the windows overlap, they are merged into a single session and the old sessions are discarded. 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, Merger, 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 the configurationparameters forcache size, andcommit interval.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 StreamsConfigvia 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- an- Initializerthat computes an initial intermediate aggregation result. Cannot be- null.
- sessionMerger- a- Mergerthat combines two aggregation results. Cannot be- null.
- Returns:
- a windowed KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key per session
 
- 
aggregateKTable<Windowed<K>,V> aggregate(Initializer<V> initializer, Merger<? super K, V> sessionMerger, Named named) Aggregate the values of records in these streams by the grouped key and defined sessions. Note that sessions are generated on a per-key basis and records with different keys create independent sessions. Records withnullkey or value are ignored. The result is written into a localSessionStore(which is basically an ever-updating materialized view). Furthermore, updates to the store are sent downstream into aKTablechangelog stream.The specified Initializeris applied directly before the first input record per session is processed to provide an initial intermediate aggregation result that is used to process the first record per session. 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. The specifiedMergeris used to merge two existing sessions into one, i.e., when the windows overlap, they are merged into a single session and the old sessions are discarded. 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, Merger, 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 the configurationparameters forcache size, andcommit interval.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 StreamsConfigvia 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- an- Initializerthat computes an initial intermediate aggregation result. Cannot be- null.
- sessionMerger- a- Mergerthat combines two aggregation results. Cannot be- null.
- named- a- Namedconfig used to name the processor in the topology. Cannot be- null.
- Returns:
- a windowed KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key per session
 
- 
aggregateKTable<Windowed<K>,V> aggregate(Initializer<V> initializer, Merger<? super K, V> sessionMerger, Materialized<K, V, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in these streams by the grouped key and defined sessions. Records withnullkey or value are ignored. The result is written into a localSessionStore(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 aKTablechangelog stream.The specified Initializeris 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 session (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. The specifiedMergeris used to merge two existing sessions into one, i.e., when the windows overlap, they are merged into a single session and the old sessions are discarded. 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 Materializedinstance. 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 theconfigurationparameters forcache size, andcommit interval.To query the local SessionStoreit 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 StoreQueryParameters<ReadOnlySessionStore<String, Long>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.sessionStore()); ReadOnlySessionStore<String,Long> localWindowStore = streams.store(storeQueryParams); String key = "some-word"; long fromTime = ...; long toTime = ...; WindowStoreIterator<Long> aggregateStore = localWindowStore.fetch(key, timeFrom, timeTo); // key must be local (application state is shared over all running Kafka Streams instances)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 will be backed by an internal changelog topic that will be created in Kafka. Therefore, the store name defined by the Materializedinstance 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 inStreamsConfigvia 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- an- Initializerthat computes an initial intermediate aggregation result. Cannot be- null.
- sessionMerger- a- Mergerthat combines two aggregation results. Cannot be- null.
- materialized- a- Materializedconfig used to materialize a state store. Cannot be- null.
- Returns:
- a windowed KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key within a window
 
- 
aggregateKTable<Windowed<K>,V> aggregate(Initializer<V> initializer, Merger<? super K, V> sessionMerger, Named named, Materialized<K, V, SessionStore<org.apache.kafka.common.utils.Bytes, byte[]>> materialized) Aggregate the values of records in these streams by the grouped key and defined sessions. Records withnullkey or value are ignored. The result is written into a localSessionStore(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 aKTablechangelog stream.The specified Initializeris 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 session (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. The specifiedMergeris used to merge two existing sessions into one, i.e., when the windows overlap, they are merged into a single session and the old sessions are discarded. 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 Materializedinstance. 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 theconfigurationparameters forcache size, andcommit interval.To query the local SessionStoreit must be obtained viaKafkaStreams.store(StoreQueryParameters)KafkaStreams#store(...)}:
 For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // some windowed aggregation on value type double Sting queryableStoreName = ... // the queryableStoreName should be the name of the store as defined by the Materialized instance StoreQueryParameters<ReadOnlySessionStore<String, Long>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.sessionStore()); ReadOnlySessionStore<String,Long> localWindowStore = streams.store(storeQueryParams); 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)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 will be backed by an internal changelog topic that will be created in Kafka. Therefore, the store name defined by the Materializedinstance 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 inStreamsConfigvia 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- an- Initializerthat computes an initial intermediate aggregation result. Cannot be- null.
- sessionMerger- a- Mergerthat combines two aggregation results. Cannot be- null.
- named- a- Namedconfig used to name the processor in the topology. Cannot be- null.
- materialized- a- Materializedconfig used to materialize a state store. Cannot be- null.
- Returns:
- a windowed KTablethat contains "update" records with unmodified keys, and values that represent the latest (rolling) aggregate for each key per session
 
 
-