All Downloads are FREE. Search and download functionalities are using the official Maven repository.

org.apache.kafka.streams.kstream.KGroupedStream Maven / Gradle / Ivy

/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements. See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License. You may obtain a copy of the License at
 *
 *    http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */
package org.apache.kafka.streams.kstream;

import org.apache.kafka.common.utils.Bytes;
import org.apache.kafka.streams.KafkaStreams;
import org.apache.kafka.streams.KeyValue;
import org.apache.kafka.streams.StoreQueryParameters;
import org.apache.kafka.streams.StreamsConfig;
import org.apache.kafka.streams.Topology;
import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
import org.apache.kafka.streams.state.TimestampedKeyValueStore;

/**
 * {@code KGroupedStream} is an abstraction of a grouped record stream of {@link KeyValue} pairs.
 * It is an intermediate representation of a {@link KStream} in order to apply an aggregation operation on the original
 * {@link KStream} records.
 * 

* It is an intermediate representation after a grouping of a {@link KStream} before an aggregation is applied to the * new partitions resulting in a {@link KTable}. *

* A {@code KGroupedStream} must be obtained from a {@link KStream} via {@link KStream#groupByKey() groupByKey()} or * {@link KStream#groupBy(KeyValueMapper) groupBy(...)}. * * @param Type of keys * @param Type of values * @see KStream */ public interface KGroupedStream { /** * Count the number of records in this stream by the grouped key. * Records with {@code null} key or value are ignored. * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view). * Furthermore, updates to the store are sent downstream into a {@link 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 {@link StreamsConfig configuration} parameters for * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store (which always will be of type {@link TimestampedKeyValueStore}) will be backed by * an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is * user-specified in {@link StreamsConfig} via parameter * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name * and "-changelog" is a fixed suffix. * Note that the internal store name may not be queryable through Interactive Queries. * * You can retrieve all generated internal topic names via {@link Topology#describe()}. * * @return a {@link KTable} that contains "update" records with unmodified keys and {@link Long} values that * represent the latest (rolling) count (i.e., number of records) for each key */ KTable count(); /** * Count the number of records in this stream by the grouped key. * Records with {@code null} key or value are ignored. * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view). * Furthermore, updates to the store are sent downstream into a {@link 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 {@link StreamsConfig configuration} parameters for * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* For failure and recovery the store (which always will be of type {@link TimestampedKeyValueStore}) will be backed by * an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is * user-specified in {@link StreamsConfig} via parameter * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name * and "-changelog" is a fixed suffix. * Note that the internal store name may not be queryable through Interactive Queries. * * You can retrieve all generated internal topic names via {@link Topology#describe()}. * * @param named a {@link Named} config used to name the processor in the topology * * @return a {@link KTable} that contains "update" records with unmodified keys and {@link Long} values that * represent the latest (rolling) count (i.e., number of records) for each key */ KTable count(final Named named); /** * Count the number of records in this stream by the grouped key. * Records with {@code null} key or value are ignored. * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) * provided by the given store name in {@code materialized}. * Furthermore, updates to the store are sent downstream into a {@link 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 {@link StreamsConfig configuration} parameters for * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via * {@link KafkaStreams#store(StoreQueryParameters) KafkaStreams#store(...)}. *

{@code
     * KafkaStreams streams = ... // counting words
     * String queryableStoreName = "storeName"; // the store name should be the name of the store as defined by the Materialized instance
     * StoreQueryParameters>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedKeyValueStore());
     * ReadOnlyKeyValueStore> localStore = streams.store(storeQueryParams);
     * K key = "some-word";
     * ValueAndTimestamp 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 {@link 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 {@link TimestampedKeyValueStore} -- regardless of what * is specified in the parameter {@code 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 {@link StreamsConfig} via parameter * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the * provide store name defined in {@code Materialized}, and "-changelog" is a fixed suffix. * * You can retrieve all generated internal topic names via {@link Topology#describe()}. * * @param materialized an instance of {@link Materialized} used to materialize a state store. Cannot be {@code null}. * Note: the valueSerde will be automatically set to {@link org.apache.kafka.common.serialization.Serdes#Long() Serdes#Long()} * if there is no valueSerde provided * @return a {@link KTable} that contains "update" records with unmodified keys and {@link Long} values that * represent the latest (rolling) count (i.e., number of records) for each key */ KTable count(final Materialized> materialized); /** * Count the number of records in this stream by the grouped key. * Records with {@code null} key or value are ignored. * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) * provided by the given store name in {@code materialized}. * Furthermore, updates to the store are sent downstream into a {@link 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 {@link StreamsConfig configuration} parameters for * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via * {@link KafkaStreams#store(StoreQueryParameters) KafkaStreams#store(...)}. *

{@code
     * KafkaStreams streams = ... // counting words
     * String queryableStoreName = "storeName"; // the store name should be the name of the store as defined by the Materialized instance
     * StoreQueryParameters>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedKeyValueStore());
     * ReadOnlyKeyValueStore> localStore = streams.store(storeQueryParams);
     * K key = "some-word";
     * ValueAndTimestamp 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 {@link 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 {@link TimestampedKeyValueStore} -- regardless of what * is specified in the parameter {@code 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 {@link StreamsConfig} via parameter * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the * provide store name defined in {@code Materialized}, and "-changelog" is a fixed suffix. * * You can retrieve all generated internal topic names via {@link Topology#describe()}. * * @param named a {@link Named} config used to name the processor in the topology * @param materialized an instance of {@link Materialized} used to materialize a state store. Cannot be {@code null}. * Note: the valueSerde will be automatically set to {@link org.apache.kafka.common.serialization.Serdes#Long() Serdes#Long()} * if there is no valueSerde provided * @return a {@link KTable} that contains "update" records with unmodified keys and {@link Long} values that * represent the latest (rolling) count (i.e., number of records) for each key */ KTable count(final Named named, final Materialized> materialized); /** * Combine the values of records in this stream by the grouped key. * Records with {@code null} key or value are ignored. * Combining implies that the type of the aggregate result is the same as the type of the input value * (c.f. {@link #aggregate(Initializer, Aggregator)}). *

* The specified {@link 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 {@link Reducer} is not applied and the new aggregate will be the record's * value as-is. * Thus, {@code reduce(Reducer)} 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 {@link StreamsConfig configuration} parameters for * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. * *

* For failure and recovery the store (which always will be of type {@link TimestampedKeyValueStore}) will be backed by * an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is * user-specified in {@link StreamsConfig} via parameter * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name * and "-changelog" is a fixed suffix. * Note that the internal store name may not be queryable through Interactive Queries. * * You can retrieve all generated internal topic names via {@link Topology#describe()}. * * @param reducer a {@link Reducer} that computes a new aggregate result. Cannot be {@code null}. * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the * latest (rolling) aggregate for each key. If the reduce function returns {@code null}, it is then interpreted as * deletion for the key, and future messages of the same key coming from upstream operators * will be handled as newly initialized value. */ KTable reduce(final Reducer reducer); /** * Combine the value of records in this stream by the grouped key. * Records with {@code null} key or value are ignored. * Combining implies that the type of the aggregate result is the same as the type of the input value * (c.f. {@link #aggregate(Initializer, Aggregator, Materialized)}). * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) * provided by the given store name in {@code materialized}. * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. *

* The specified {@link Reducer} is applied for each input record and computes a new aggregate using the current * aggregate (first argument) and the record's value (second argument): *

{@code
     * // At the example of a Reducer
     * new Reducer() {
     *   public Long apply(Long aggValue, Long currValue) {
     *     return aggValue + currValue;
     *   }
     * }
     * }
*

* If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's * value as-is. * Thus, {@code reduce(Reducer, Materialized)} 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 {@link StreamsConfig configuration} parameters for * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via * {@link KafkaStreams#store(StoreQueryParameters) KafkaStreams#store(...)}. *

{@code
     * KafkaStreams streams = ... // compute sum
     * String queryableStoreName = "storeName" // the store name should be the name of the store as defined by the Materialized instance
     * StoreQueryParameters>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedKeyValueStore());
     * ReadOnlyKeyValueStore> localStore = streams.store(storeQueryParams);
     * K key = "some-key";
     * ValueAndTimestamp reduceForKey = 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 {@link 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 {@link TimestampedKeyValueStore} -- regardless of what * is specified in the parameter {@code materialized}) will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is * user-specified in {@link StreamsConfig} via parameter * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name * and "-changelog" is a fixed suffix. * Note that the internal store name may not be queryable through Interactive Queries. * * You can retrieve all generated internal topic names via {@link Topology#describe()}. * * @param reducer a {@link Reducer} that computes a new aggregate result. Cannot be {@code null}. * @param materialized an instance of {@link Materialized} used to materialize a state store. Cannot be {@code null}. * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the * latest (rolling) aggregate for each key */ KTable reduce(final Reducer reducer, final Materialized> materialized); /** * Combine the value of records in this stream by the grouped key. * Records with {@code null} key or value are ignored. * Combining implies that the type of the aggregate result is the same as the type of the input value * (c.f. {@link #aggregate(Initializer, Aggregator, Materialized)}). * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) * provided by the given store name in {@code materialized}. * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. *

* The specified {@link Reducer} is applied for each input record and computes a new aggregate using the current * aggregate (first argument) and the record's value (second argument): *

{@code
     * // At the example of a Reducer
     * new Reducer() {
     *   public Long apply(Long aggValue, Long currValue) {
     *     return aggValue + currValue;
     *   }
     * }
     * }
*

* If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's * value as-is. * Thus, {@code reduce(Reducer, Materialized)} 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 {@link StreamsConfig configuration} parameters for * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via * {@link KafkaStreams#store(StoreQueryParameters) KafkaStreams#store(...)}. *

{@code
     * KafkaStreams streams = ... // compute sum
     * String queryableStoreName = "storeName" // the store name should be the name of the store as defined by the Materialized instance
     * StoreQueryParameters>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedKeyValueStore());
     * ReadOnlyKeyValueStore> localStore = streams.store(storeQueryParams);
     * K key = "some-key";
     * ValueAndTimestamp reduceForKey = 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 {@link 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 {@link TimestampedKeyValueStore} -- regardless of what * is specified in the parameter {@code materialized}) will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is * user-specified in {@link StreamsConfig} via parameter * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name * and "-changelog" is a fixed suffix. * Note that the internal store name may not be queryable through Interactive Queries. * * You can retrieve all generated internal topic names via {@link Topology#describe()}. * * @param reducer a {@link Reducer} that computes a new aggregate result. Cannot be {@code null}. * @param named a {@link Named} config used to name the processor in the topology. * @param materialized an instance of {@link Materialized} used to materialize a state store. Cannot be {@code null}. * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the * latest (rolling) aggregate for each key. If the reduce function returns {@code null}, it is then interpreted as * deletion for the key, and future messages of the same key coming from upstream operators * will be handled as newly initialized value. */ KTable reduce(final Reducer reducer, final Named named, final Materialized> materialized); /** * Aggregate the values of records in this stream by the grouped key. * Records with {@code null} key or value are ignored. * Aggregating is a generalization of {@link #reduce(Reducer) combining via reduce(...)} as it, for example, * allows the result to have a different type than the input values. *

* The specified {@link Initializer} is applied once directly before the first input record is processed to * provide an initial intermediate aggregation result that is used to process the first record. * The specified {@link Aggregator} is applied for each input record and computes a new aggregate using the current * aggregate (or for the very first record using the intermediate aggregation result provided via the * {@link Initializer}) and the record's value. * Thus, {@code aggregate(Initializer, Aggregator)} can be used to compute aggregate functions like * count (c.f. {@link #count()}). *

* The default value serde from config will be used for serializing the result. * If a different serde is required then you should use {@link #aggregate(Initializer, Aggregator, Materialized)}. *

* Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. * *

* For failure and recovery the store (which always will be of type {@link TimestampedKeyValueStore}) will be backed by * an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${internalStoreName}-changelog", where "applicationId" is * user-specified in {@link StreamsConfig} via parameter * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "internalStoreName" is an internal name * and "-changelog" is a fixed suffix. * Note that the internal store name may not be queryable through Interactive Queries. * * You can retrieve all generated internal topic names via {@link Topology#describe()}. * * @param initializer an {@link Initializer} that computes an initial intermediate aggregation result * @param aggregator an {@link Aggregator} that computes a new aggregate result * @param the value type of the resulting {@link KTable} * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the * latest (rolling) aggregate for each key. If the aggregate function returns {@code null}, it is then interpreted as * deletion for the key, and future messages of the same key coming from upstream operators * will be handled as newly initialized value. */ KTable aggregate(final Initializer initializer, final Aggregator aggregator); /** * Aggregate the values of records in this stream by the grouped key. * Records with {@code null} key or value are ignored. * Aggregating is a generalization of {@link #reduce(Reducer) combining via reduce(...)} as it, for example, * allows the result to have a different type than the input values. * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) * that can be queried by the given store name in {@code materialized}. * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. *

* The specified {@link Initializer} is applied once directly before the first input record is processed to * provide an initial intermediate aggregation result that is used to process the first record. * The specified {@link Aggregator} is applied for each input record and computes a new aggregate using the current * aggregate (or for the very first record using the intermediate aggregation result provided via the * {@link Initializer}) and the record's value. * Thus, {@code aggregate(Initializer, Aggregator, Materialized)} can be used to compute aggregate functions like * count (c.f. {@link #count()}). *

* Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via * {@link KafkaStreams#store(StoreQueryParameters) KafkaStreams#store(...)}: *

{@code
     * 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
     * StoreQueryParameters>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedKeyValueStore());
     * ReadOnlyKeyValueStore> localStore = streams.store(storeQueryParams);
     * K key = "some-key";
     * ValueAndTimestamp 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 {@link 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 {@link TimestampedKeyValueStore} -- regardless of what * is specified in the parameter {@code 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 {@link StreamsConfig} via parameter * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the * provide store name defined in {@code Materialized}, and "-changelog" is a fixed suffix. * * You can retrieve all generated internal topic names via {@link Topology#describe()}. * * @param initializer an {@link Initializer} that computes an initial intermediate aggregation result * @param aggregator an {@link Aggregator} that computes a new aggregate result * @param materialized an instance of {@link Materialized} used to materialize a state store. Cannot be {@code null}. * @param the value type of the resulting {@link KTable} * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the * latest (rolling) aggregate for each key */ KTable aggregate(final Initializer initializer, final Aggregator aggregator, final Materialized> materialized); /** * Aggregate the values of records in this stream by the grouped key. * Records with {@code null} key or value are ignored. * Aggregating is a generalization of {@link #reduce(Reducer) combining via reduce(...)} as it, for example, * allows the result to have a different type than the input values. * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) * that can be queried by the given store name in {@code materialized}. * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. *

* The specified {@link Initializer} is applied once directly before the first input record is processed to * provide an initial intermediate aggregation result that is used to process the first record. * The specified {@link Aggregator} is applied for each input record and computes a new aggregate using the current * aggregate (or for the very first record using the intermediate aggregation result provided via the * {@link Initializer}) and the record's value. * Thus, {@code aggregate(Initializer, Aggregator, Materialized)} can be used to compute aggregate functions like * count (c.f. {@link #count()}). *

* Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for * {@link StreamsConfig#STATESTORE_CACHE_MAX_BYTES_CONFIG cache size}, and * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit interval}. *

* To query the local {@link ReadOnlyKeyValueStore} it must be obtained via * {@link KafkaStreams#store(StoreQueryParameters) KafkaStreams#store(...)}: *

{@code
     * 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
     * StoreQueryParameters>> storeQueryParams = StoreQueryParameters.fromNameAndType(queryableStoreName, QueryableStoreTypes.timestampedKeyValueStore());
     * ReadOnlyKeyValueStore> localStore = streams.store(storeQueryParams);
     * K key = "some-key";
     * ValueAndTimestamp 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 {@link 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 {@link TimestampedKeyValueStore} -- regardless of what * is specified in the parameter {@code 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 {@link StreamsConfig} via parameter * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the * provide store name defined in {@code Materialized}, and "-changelog" is a fixed suffix. * * You can retrieve all generated internal topic names via {@link Topology#describe()}. * * @param initializer an {@link Initializer} that computes an initial intermediate aggregation result * @param aggregator an {@link Aggregator} that computes a new aggregate result * @param named a {@link Named} config used to name the processor in the topology * @param materialized an instance of {@link Materialized} used to materialize a state store. Cannot be {@code null}. * @param the value type of the resulting {@link KTable} * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the * latest (rolling) aggregate for each key. If the aggregate function returns {@code null}, it is then interpreted as * deletion for the key, and future messages of the same key coming from upstream operators * will be handled as newly initialized value. */ KTable aggregate(final Initializer initializer, final Aggregator aggregator, final Named named, final Materialized> materialized); /** * Create a new {@link TimeWindowedKStream} instance that can be used to perform windowed aggregations. * @param windows the specification of the aggregation {@link Windows} * @param the window type * @return an instance of {@link TimeWindowedKStream} */ TimeWindowedKStream windowedBy(final Windows windows); /** * Create a new {@link TimeWindowedKStream} instance that can be used to perform sliding windowed aggregations. * @param windows the specification of the aggregation {@link SlidingWindows} * @return an instance of {@link TimeWindowedKStream} */ TimeWindowedKStream windowedBy(final SlidingWindows windows); /** * Create a new {@link SessionWindowedKStream} instance that can be used to perform session windowed aggregations. * @param windows the specification of the aggregation {@link SessionWindows} * @return an instance of {@link TimeWindowedKStream} */ SessionWindowedKStream windowedBy(final SessionWindows windows); /** * Create a new {@link CogroupedKStream} from the this grouped KStream to allow cogrouping other * {@code KGroupedStream} to it. * {@link CogroupedKStream} is an abstraction of multiple grouped record streams of {@link KeyValue} pairs. * It is an intermediate representation after a grouping of {@link KStream}s, before the * aggregations are applied to the new partitions resulting in a {@link KTable}. *

* The specified {@link Aggregator} is applied in the actual {@link CogroupedKStream#aggregate(Initializer) * 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 {@link Initializer} that * is passed into {@link CogroupedKStream#aggregate(Initializer)}) and the record's value. * * @param aggregator an {@link Aggregator} that computes a new aggregate result * @param the type of the output values * @return a {@link CogroupedKStream} */ CogroupedKStream cogroup(final Aggregator aggregator); }





© 2015 - 2024 Weber Informatics LLC | Privacy Policy