Interface KTable<K,V>
-
- Type Parameters:
K- Type of primary keysV- Type of value changes
@Evolving public interface KTable<K,V>KTableis an abstraction of a changelog stream from a primary-keyed table. Each record in this changelog stream is an update on the primary-keyed table with the record key as the primary key.A
KTableis eitherdefined from a single Kafka topicthat is consumed message by message or the result of aKTabletransformation. An aggregation of aKStreamalso yields aKTable.A
KTablecan be transformed record by record, joined with anotherKTableorKStream, or can be re-partitioned and aggregated into a newKTable.Some
KTables have an internal state (aReadOnlyKeyValueStore) and are therefore queryable via the interactive queries API. For example:final KTable table = ... ... final KafkaStreams streams = ...; streams.start() ... final String queryableStoreName = table.queryableStoreName(); // returns null if KTable is not queryable ReadOnlyKeyValueStore view = streams.store(queryableStoreName, QueryableStoreTypes.keyValueStore()); view.get(key);Records from the source topic that have null keys are dropped.
- See Also:
KStream,KGroupedTable,GlobalKTable,StreamsBuilder.table(String)
-
-
Method Summary
All Methods Instance Methods Abstract Methods Deprecated Methods Modifier and Type Method Description KTable<K,V>filter(Predicate<? super K,? super V> predicate)Create a newKTablethat consists of all records of thisKTablewhich satisfy the given predicate.KTable<K,V>filter(Predicate<? super K,? super V> predicate, java.lang.String queryableStoreName)Deprecated.KTable<K,V>filter(Predicate<? super K,? super V> predicate, Materialized<K,V,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)Create a newKTablethat consists of all records of thisKTablewhich satisfy the given predicate.KTable<K,V>filter(Predicate<? super K,? super V> predicate, StateStoreSupplier<KeyValueStore> storeSupplier)Deprecated.KTable<K,V>filterNot(Predicate<? super K,? super V> predicate)Create a newKTablethat consists all records of thisKTablewhich do not satisfy the given predicate.KTable<K,V>filterNot(Predicate<? super K,? super V> predicate, java.lang.String queryableStoreName)Deprecated.KTable<K,V>filterNot(Predicate<? super K,? super V> predicate, Materialized<K,V,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)Create a newKTablethat consists all records of thisKTablewhich do not satisfy the given predicate.KTable<K,V>filterNot(Predicate<? super K,? super V> predicate, StateStoreSupplier<KeyValueStore> storeSupplier)Deprecated.voidforeach(ForeachAction<? super K,? super V> action)Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable.<KR,VR>
KGroupedTable<KR,VR>groupBy(KeyValueMapper<? super K,? super V,KeyValue<KR,VR>> selector)Re-groups the records of thisKTableusing the providedKeyValueMapperand default serializers and deserializers.<KR,VR>
KGroupedTable<KR,VR>groupBy(KeyValueMapper<? super K,? super V,KeyValue<KR,VR>> selector, org.apache.kafka.common.serialization.Serde<KR> keySerde, org.apache.kafka.common.serialization.Serde<VR> valueSerde)Deprecated.<KR,VR>
KGroupedTable<KR,VR>groupBy(KeyValueMapper<? super K,? super V,KeyValue<KR,VR>> selector, Serialized<KR,VR> serialized)Re-groups the records of thisKTableusing the providedKeyValueMapperandSerdes as specified bySerialized.<VO,VR>
KTable<K,VR>join(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner)Join records of thisKTablewith anotherKTable's records using non-windowed inner equi join.<VO,VR>
KTable<K,VR>join(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, org.apache.kafka.common.serialization.Serde<VR> joinSerde, java.lang.String queryableStoreName)<VO,VR>
KTable<K,VR>join(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, Materialized<K,VR,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)Join records of thisKTablewith anotherKTable's records using non-windowed inner equi join.<VO,VR>
KTable<K,VR>join(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, StateStoreSupplier<KeyValueStore> storeSupplier)Deprecated.<VO,VR>
KTable<K,VR>leftJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner)Join records of thisKTable(left input) with anotherKTable's (right input) records using non-windowed left equi join.<VO,VR>
KTable<K,VR>leftJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, org.apache.kafka.common.serialization.Serde<VR> joinSerde, java.lang.String queryableStoreName)<VO,VR>
KTable<K,VR>leftJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, Materialized<K,VR,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)Join records of thisKTable(left input) with anotherKTable's (right input) records using non-windowed left equi join.<VO,VR>
KTable<K,VR>leftJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, StateStoreSupplier<KeyValueStore> storeSupplier)<VR> KTable<K,VR>mapValues(ValueMapper<? super V,? extends VR> mapper)Create a newKTableby transforming the value of each record in thisKTableinto a new value (with possible new type) in the newKTable.<VR> KTable<K,VR>mapValues(ValueMapper<? super V,? extends VR> mapper, org.apache.kafka.common.serialization.Serde<VR> valueSerde, java.lang.String queryableStoreName)<VR> KTable<K,VR>mapValues(ValueMapper<? super V,? extends VR> mapper, org.apache.kafka.common.serialization.Serde<VR> valueSerde, StateStoreSupplier<KeyValueStore> storeSupplier)<VR> KTable<K,VR>mapValues(ValueMapper<? super V,? extends VR> mapper, Materialized<K,VR,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)Create a newKTableby transforming the value of each record in thisKTableinto a new value (with possible new type) in the newKTable.<VR> KTable<K,VR>mapValues(ValueMapperWithKey<? super K,? super V,? extends VR> mapper)Create a newKTableby transforming the value of each record in thisKTableinto a new value (with possible new type) in the newKTable.<VR> KTable<K,VR>mapValues(ValueMapperWithKey<? super K,? super V,? extends VR> mapper, Materialized<K,VR,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)Create a newKTableby transforming the value of each record in thisKTableinto a new value (with possible new type) in the newKTable.<VO,VR>
KTable<K,VR>outerJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner)Join records of thisKTable(left input) with anotherKTable's (right input) records using non-windowed outer equi join.<VO,VR>
KTable<K,VR>outerJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, org.apache.kafka.common.serialization.Serde<VR> joinSerde, java.lang.String queryableStoreName)<VO,VR>
KTable<K,VR>outerJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, Materialized<K,VR,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)Join records of thisKTable(left input) with anotherKTable's (right input) records using non-windowed outer equi join.<VO,VR>
KTable<K,VR>outerJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, StateStoreSupplier<KeyValueStore> storeSupplier)voidprint()Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable.voidprint(java.lang.String label)Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable.voidprint(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde)Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable.voidprint(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, java.lang.String label)Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable.java.lang.StringqueryableStoreName()Get the name of the local state store used that can be used to query thisKTable.KTable<K,V>through(java.lang.String topic)Deprecated.KTable<K,V>through(java.lang.String topic, java.lang.String queryableStoreName)Deprecated.usetoStream()followed byto(topic)andStreamsBuilder#table(topic, Materialized.as(queryableStoreName))to read back as aKTableKTable<K,V>through(java.lang.String topic, StateStoreSupplier<KeyValueStore> storeSupplier)Deprecated.usetoStream()followed byto(topic)and andStreamsBuilder#table(topic, Materialized.as(KeyValueBytesStoreSupplier))to read back as aKTableKTable<K,V>through(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, java.lang.String topic)Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde))andStreamsBuilder#table(topic)to read back as aKTableKTable<K,V>through(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, java.lang.String topic, java.lang.String queryableStoreName)Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde))andStreamsBuilder#table(topic, Materialized.as(queryableStoreName))to read back as aKTableKTable<K,V>through(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, java.lang.String topic, StateStoreSupplier<KeyValueStore> storeSupplier)Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde))andStreamsBuilder#table(topic, Materialized.as(KeyValueBytesStoreSupplier)to read back as aKTableKTable<K,V>through(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic)Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde, partitioner))andStreamsBuilder#table(topic)to read back as aKTableKTable<K,V>through(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic, java.lang.String queryableStoreName)Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde, partitioner))andStreamsBuilder#table(topic, Materialized.as(queryableStoreName))to read back as aKTableKTable<K,V>through(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic, StateStoreSupplier<KeyValueStore> storeSupplier)Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde, partitioner))andStreamsBuilder#table(topic, Materialized.as(KeyValueBytesStoreSupplier))to read back as aKTableKTable<K,V>through(StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic)Deprecated.usetoStream()followed byto(topic, Produced.streamPartitioner(partitioner))andStreamsBuilder#table(topic)to read back as aKTableKTable<K,V>through(StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic, java.lang.String queryableStoreName)Deprecated.usetoStream()followed byto(topic, Produced.streamPartitioner(partitioner))andStreamsBuilder#table(topic, Materialized.as(queryableStoreName))to read back as aKTableKTable<K,V>through(StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic, StateStoreSupplier<KeyValueStore> storeSupplier)Deprecated.usetoStream()followed byto(topic, Produced.streamPartitioner(partitioner))andStreamsBuilder#table(topic, Materialized.as(KeyValueBytesStoreSupplier)to read back as aKTablevoidto(java.lang.String topic)Deprecated.usetoStream()followed byto(topic)voidto(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, java.lang.String topic)Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde)voidto(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic)Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde, partioner)voidto(StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic)Deprecated.usetoStream()followed byto(topic, Produced.withStreamPartitioner(partitioner)KStream<K,V>toStream()Convert this changelog stream to aKStream.<KR> KStream<KR,V>toStream(KeyValueMapper<? super K,? super V,? extends KR> mapper)Convert this changelog stream to aKStreamusing the givenKeyValueMapperto select the new key.voidwriteAsText(java.lang.String filePath)Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable.voidwriteAsText(java.lang.String filePath, java.lang.String label)Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable.voidwriteAsText(java.lang.String filePath, java.lang.String label, org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde)Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable.voidwriteAsText(java.lang.String filePath, org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde)Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable.
-
-
-
Method Detail
-
filter
KTable<K,V> filter(Predicate<? super K,? super V> predicate)
Create a newKTablethat consists of all records of thisKTablewhich satisfy the given predicate. All records that do not satisfy the predicate are dropped. For eachKTableupdate, the filter is evaluated based on the current update record and then an update record is produced for the resultKTable. This is a stateless record-by-record operation.Note that
filterfor a changelog stream works differently thanrecord stream filters, becauserecordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded directly if required (i.e., if there is anything to be deleted). Furthermore, for each record that gets dropped (i.e., does not satisfy the given predicate) a tombstone record is forwarded.- Parameters:
predicate- a filterPredicatethat is applied to each record- Returns:
- a
KTablethat contains only those records that satisfy the given predicate - See Also:
filterNot(Predicate)
-
filter
KTable<K,V> filter(Predicate<? super K,? super V> predicate, Materialized<K,V,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Create a newKTablethat consists of all records of thisKTablewhich satisfy the given predicate. All records that do not satisfy the predicate are dropped. For eachKTableupdate, the filter is evaluated based on the current update record and then an update record is produced for the resultKTable. This is a stateless record-by-record operation.Note that
filterfor a changelog stream works differently thanrecord stream filters, becauserecordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded directly if required (i.e., if there is anything to be deleted). Furthermore, for each record that gets dropped (i.e., does not satisfy the given predicate) a tombstone record is forwarded.To query the local
KeyValueStoreit must be obtained viaKafkaStreams#store(...):
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // filtering words ReadOnlyKeyValueStore<K,V> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, V>keyValueStore()); K key = "some-word"; V valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)KafkaStreams.allMetadata()to query the value of the key on a parallel running instance of your Kafka Streams application. The store name to query with is specified byMaterialized.as(String)orMaterialized.as(KeyValueBytesStoreSupplier).- Parameters:
predicate- a filterPredicatethat is applied to each recordmaterialized- aMaterializedthat describes how theStateStorefor the resultingKTableshould be materialized. Cannot benull- Returns:
- a
KTablethat contains only those records that satisfy the given predicate - See Also:
filterNot(Predicate, Materialized)
-
filter
@Deprecated KTable<K,V> filter(Predicate<? super K,? super V> predicate, java.lang.String queryableStoreName)
Deprecated.Create a newKTablethat consists of all records of thisKTablewhich satisfy the given predicate. All records that do not satisfy the predicate are dropped. For eachKTableupdate the filter is evaluated on the updated record to produce an updated record for the resultKTable. This is a stateless record-by-record operation.Note that
filterfor a changelog stream works differently thanrecord stream filters, becauserecordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded directly if required (i.e., if there is anything to be deleted). Furthermore, for each record that gets dropped (i.e., does not satisfy the given predicate) a tombstone record is forwarded.To query the local
KeyValueStoreit must be obtained viaKafkaStreams#store(...):
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // filtering words ReadOnlyKeyValueStore<K,V> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, V>keyValueStore()); K key = "some-word"; V valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)KafkaStreams.allMetadata()to query the value of the key on a parallel running instance of your Kafka Streams application.- Parameters:
predicate- a filterPredicatethat is applied to each recordqueryableStoreName- a user-provided name of the underlyingKTablethat can be used to subsequently query the operation results; valid characters are ASCII alphanumerics, '.', '_' and '-'. Ifnullthen the results cannot be queried (i.e., that would be equivalent to callingfilter(Predicate).- Returns:
- a
KTablethat contains only those records that satisfy the given predicate - See Also:
filterNot(Predicate, Materialized)
-
filter
@Deprecated KTable<K,V> filter(Predicate<? super K,? super V> predicate, StateStoreSupplier<KeyValueStore> storeSupplier)
Deprecated.Create a newKTablethat consists of all records of thisKTablewhich satisfy the given predicate. All records that do not satisfy the predicate are dropped. For eachKTableupdate the filter is evaluated on the updated record to produce an updated record for the resultKTable. This is a stateless record-by-record operation.Note that
filterfor a changelog stream works differently thanrecord stream filters, becauserecordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded directly if required (i.e., if there is anything to be deleted). Furthermore, for each record that gets dropped (i.e., does not satisfy the given predicate) a tombstone record is forwarded.To query the local
KeyValueStoreit must be obtained viaKafkaStreams#store(...):
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // filtering words ReadOnlyKeyValueStore<K,V> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, V>keyValueStore()); K key = "some-word"; V valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)KafkaStreams.allMetadata()to query the value of the key on a parallel running instance of your Kafka Streams application.- Parameters:
predicate- a filterPredicatethat is applied to each recordstoreSupplier- user defined state store supplier. Cannot benull.- Returns:
- a
KTablethat contains only those records that satisfy the given predicate - See Also:
filterNot(Predicate, Materialized)
-
filterNot
KTable<K,V> filterNot(Predicate<? super K,? super V> predicate)
Create a newKTablethat consists all records of thisKTablewhich do not satisfy the given predicate. All records that do satisfy the predicate are dropped. For eachKTableupdate, the filter is evaluated based on the current update record and then an update record is produced for the resultKTable. This is a stateless record-by-record operation.Note that
filterNotfor a changelog stream works differently thanrecord stream filters, becauserecordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded directly if required (i.e., if there is anything to be deleted). Furthermore, for each record that gets dropped (i.e., does satisfy the given predicate) a tombstone record is forwarded.- Parameters:
predicate- a filterPredicatethat is applied to each record- Returns:
- a
KTablethat contains only those records that do not satisfy the given predicate - See Also:
filter(Predicate)
-
filterNot
KTable<K,V> filterNot(Predicate<? super K,? super V> predicate, Materialized<K,V,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Create a newKTablethat consists all records of thisKTablewhich do not satisfy the given predicate. All records that do satisfy the predicate are dropped. For eachKTableupdate, the filter is evaluated based on the current update record and then an update record is produced for the resultKTable. This is a stateless record-by-record operation.Note that
filterNotfor a changelog stream works differently thanrecord stream filters, becauserecordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded directly if required (i.e., if there is anything to be deleted). Furthermore, for each record that gets dropped (i.e., does satisfy the given predicate) a tombstone record is forwarded.To query the local
KeyValueStoreit must be obtained viaKafkaStreams#store(...):
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // filtering words ReadOnlyKeyValueStore<K,V> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, V>keyValueStore()); K key = "some-word"; V valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)KafkaStreams.allMetadata()to query the value of the key on a parallel running instance of your Kafka Streams application. The store name to query with is specified byMaterialized.as(String)orMaterialized.as(KeyValueBytesStoreSupplier).- Parameters:
predicate- a filterPredicatethat is applied to each recordmaterialized- aMaterializedthat describes how theStateStorefor the resultingKTableshould be materialized. Cannot benull- Returns:
- a
KTablethat contains only those records that do not satisfy the given predicate - See Also:
filter(Predicate, Materialized)
-
filterNot
@Deprecated KTable<K,V> filterNot(Predicate<? super K,? super V> predicate, StateStoreSupplier<KeyValueStore> storeSupplier)
Deprecated.Create a newKTablethat consists all records of thisKTablewhich do not satisfy the given predicate. All records that do satisfy the predicate are dropped. For eachKTableupdate the filter is evaluated on the updated record to produce an updated record for the resultKTable. This is a stateless record-by-record operation.Note that
filterNotfor a changelog stream works differently thanrecord stream filters, becauserecordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded directly if required (i.e., if there is anything to be deleted). Furthermore, for each record that gets dropped (i.e., does satisfy the given predicate) a tombstone record is forwarded.To query the local
KeyValueStoreit must be obtained viaKafkaStreams#store(...):
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // filtering words ReadOnlyKeyValueStore<K,V> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, V>keyValueStore()); K key = "some-word"; V valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)KafkaStreams.allMetadata()to query the value of the key on a parallel running instance of your Kafka Streams application.- Parameters:
predicate- a filterPredicatethat is applied to each recordstoreSupplier- user defined state store supplier. Cannot benull.- Returns:
- a
KTablethat contains only those records that do not satisfy the given predicate - See Also:
filter(Predicate, Materialized)
-
filterNot
@Deprecated KTable<K,V> filterNot(Predicate<? super K,? super V> predicate, java.lang.String queryableStoreName)
Deprecated.Create a newKTablethat consists all records of thisKTablewhich do not satisfy the given predicate. All records that do satisfy the predicate are dropped. For eachKTableupdate the filter is evaluated on the updated record to produce an updated record for the resultKTable. This is a stateless record-by-record operation.Note that
filterNotfor a changelog stream works differently thanrecord stream filters, becauserecordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded directly if required (i.e., if there is anything to be deleted). Furthermore, for each record that gets dropped (i.e., does satisfy the given predicate) a tombstone record is forwarded.To query the local
KeyValueStoreit must be obtained viaKafkaStreams#store(...):
For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams streams = ... // filtering words ReadOnlyKeyValueStore<K,V> localStore = streams.store(queryableStoreName, QueryableStoreTypes.<K, V>keyValueStore()); K key = "some-word"; V valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)KafkaStreams.allMetadata()to query the value of the key on a parallel running instance of your Kafka Streams application.- Parameters:
predicate- a filterPredicatethat is applied to each recordqueryableStoreName- a user-provided name of the underlyingKTablethat can be used to subsequently query the operation results; valid characters are ASCII alphanumerics, '.', '_' and '-'. Ifnullthen the results cannot be queried (i.e., that would be equivalent to callingfilterNot(Predicate).- Returns:
- a
KTablethat contains only those records that do not satisfy the given predicate - See Also:
filter(Predicate, Materialized)
-
mapValues
<VR> KTable<K,VR> mapValues(ValueMapper<? super V,? extends VR> mapper)
Create a newKTableby transforming the value of each record in thisKTableinto a new value (with possible new type) in the newKTable. For eachKTableupdate the providedValueMapperis applied to the value of the updated record and computes a new value for it, resulting in an updated record for the resultKTable. Thus, an input record<K,V>can be transformed into an output record<K:V'>. This is a stateless record-by-record operation.The example below counts the number of token of the value string.
KTable<String, String> inputTable = builder.table("topic"); KTable<String, Integer> outputTable = inputTable.mapValue(new ValueMapper<String, Integer> { Integer apply(String value) { return value.split(" ").length; } });This operation preserves data co-location with respect to the key. Thus, no internal data redistribution is required if a key based operator (like a join) is applied to the result
KTable.Note that
mapValuesfor a changelog stream works differently thanrecord stream filters, becauserecordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to delete the corresponding record in the resultKTable.- Type Parameters:
VR- the value type of the resultKTable- Parameters:
mapper- aValueMapperthat computes a new output value- Returns:
- a
KTablethat contains records with unmodified keys and new values (possibly of different type)
-
mapValues
<VR> KTable<K,VR> mapValues(ValueMapperWithKey<? super K,? super V,? extends VR> mapper)
Create a newKTableby transforming the value of each record in thisKTableinto a new value (with possible new type) in the newKTable. For eachKTableupdate the providedValueMapperWithKeyis applied to the value of the update record and computes a new value for it, resulting in an updated record for the resultKTable. Thus, an input record<K,V>can be transformed into an output record<K:V'>. This is a stateless record-by-record operation.The example below counts the number of token of value and key strings.
KTable<String, String> inputTable = builder.table("topic"); KTable<String, Integer> outputTable = inputTable.mapValue(new ValueMapperWithKey<String, String, Integer> { Integer apply(String readOnlyKey, String value) { return readOnlyKey.split(" ").length + value.split(" ").length; } });Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning. This operation preserves data co-location with respect to the key. Thus, no internal data redistribution is required if a key based operator (like a join) is applied to the result
KTable.Note that
mapValuesfor a changelog stream works differently thanrecord stream filters, becauserecordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to delete the corresponding record in the resultKTable.- Type Parameters:
VR- the value type of the resultKTable- Parameters:
mapper- aValueMapperWithKeythat computes a new output value- Returns:
- a
KTablethat contains records with unmodified keys and new values (possibly of different type)
-
mapValues
<VR> KTable<K,VR> mapValues(ValueMapper<? super V,? extends VR> mapper, Materialized<K,VR,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Create a newKTableby transforming the value of each record in thisKTableinto a new value (with possible new type) in the newKTable. For eachKTableupdate the providedValueMapperis applied to the value of the updated record and computes a new value for it, resulting in an updated record for the resultKTable. Thus, an input record<K,V>can be transformed into an output record<K:V'>. This is a stateless record-by-record operation.The example below counts the number of token of the value string.
KTable<String, String> inputTable = builder.table("topic"); KTable<String, Integer> outputTable = inputTable.mapValue(new ValueMapper<String, Integer> { Integer apply(String value) { return value.split(" ").length; } });To query the local
KeyValueStorerepresenting outputTable above it must be obtained viaKafkaStreams#store(...): For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams.allMetadata()to query the value of the key on a parallel running instance of your Kafka Streams application. The store name to query with is specified byMaterialized.as(String)orMaterialized.as(KeyValueBytesStoreSupplier).This operation preserves data co-location with respect to the key. Thus, no internal data redistribution is required if a key based operator (like a join) is applied to the result
KTable.Note that
mapValuesfor a changelog stream works differently thanrecord stream filters, becauserecordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to delete the corresponding record in the resultKTable.- Type Parameters:
VR- the value type of the resultKTable- Parameters:
mapper- aValueMapperthat computes a new output valuematerialized- aMaterializedthat describes how theStateStorefor the resultingKTableshould be materialized. Cannot benull- Returns:
- a
KTablethat contains records with unmodified keys and new values (possibly of different type)
-
mapValues
<VR> KTable<K,VR> mapValues(ValueMapperWithKey<? super K,? super V,? extends VR> mapper, Materialized<K,VR,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Create a newKTableby transforming the value of each record in thisKTableinto a new value (with possible new type) in the newKTable. For eachKTableupdate the providedValueMapperWithKeyis applied to the value of the update record and computes a new value for it, resulting in an updated record for the resultKTable. Thus, an input record<K,V>can be transformed into an output record<K:V'>. This is a stateless record-by-record operation.The example below counts the number of token of value and key strings.
KTable<String, String> inputTable = builder.table("topic"); KTable<String, Integer> outputTable = inputTable.mapValue(new ValueMapperWithKey<String, String, Integer> { Integer apply(String readOnlyKey, String value) { return readOnlyKey.split(" ").length + value.split(" ").length; } });To query the local
KeyValueStorerepresenting outputTable above it must be obtained viaKafkaStreams#store(...): For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams.allMetadata()to query the value of the key on a parallel running instance of your Kafka Streams application. The store name to query with is specified byMaterialized.as(String)orMaterialized.as(KeyValueBytesStoreSupplier).Note that the key is read-only and should not be modified, as this can lead to corrupt partitioning. This operation preserves data co-location with respect to the key. Thus, no internal data redistribution is required if a key based operator (like a join) is applied to the result
KTable.Note that
mapValuesfor a changelog stream works differently thanrecord stream filters, becauserecordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to delete the corresponding record in the resultKTable.- Type Parameters:
VR- the value type of the resultKTable- Parameters:
mapper- aValueMapperWithKeythat computes a new output valuematerialized- aMaterializedthat describes how theStateStorefor the resultingKTableshould be materialized. Cannot benull- Returns:
- a
KTablethat contains records with unmodified keys and new values (possibly of different type)
-
mapValues
@Deprecated <VR> KTable<K,VR> mapValues(ValueMapper<? super V,? extends VR> mapper, org.apache.kafka.common.serialization.Serde<VR> valueSerde, java.lang.String queryableStoreName)
Create a newKTableby transforming the value of each record in thisKTableinto a new value (with possible new type) in the newKTable. For eachKTableupdate the providedValueMapperis applied to the value of the updated record and computes a new value for it, resulting in an updated record for the resultKTable. Thus, an input record<K,V>can be transformed into an output record<K:V'>. This is a stateless record-by-record operation.The example below counts the number of token of the value string.
KTable<String, String> inputTable = builder.table("topic"); KTable<String, Integer> outputTable = inputTable.mapValue(new ValueMapper<String, Integer> { Integer apply(String value) { return value.split(" ").length; } });To query the local
KeyValueStorerepresenting outputTable above it must be obtained viaKafkaStreams#store(...): For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams.allMetadata()to query the value of the key on a parallel running instance of your Kafka Streams application.This operation preserves data co-location with respect to the key. Thus, no internal data redistribution is required if a key based operator (like a join) is applied to the result
KTable.Note that
mapValuesfor a changelog stream works differently thanrecord stream filters, becauserecordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to delete the corresponding record in the resultKTable.- Type Parameters:
VR- the value type of the resultKTable- Parameters:
mapper- aValueMapperthat computes a new output valuequeryableStoreName- a user-provided name of the underlyingKTablethat can be used to subsequently query the operation results; valid characters are ASCII alphanumerics, '.', '_' and '-'. Ifnullthen the results cannot be queried (i.e., that would be equivalent to callingmapValues(ValueMapper).valueSerde- serializer for new value type- Returns:
- a
KTablethat contains records with unmodified keys and new values (possibly of different type)
-
mapValues
@Deprecated <VR> KTable<K,VR> mapValues(ValueMapper<? super V,? extends VR> mapper, org.apache.kafka.common.serialization.Serde<VR> valueSerde, StateStoreSupplier<KeyValueStore> storeSupplier)
Deprecated.Create a newKTableby transforming the value of each record in thisKTableinto a new value (with possible new type) in the newKTable. For eachKTableupdate the providedValueMapperis applied to the value of the updated record and computes a new value for it, resulting in an updated record for the resultKTable. Thus, an input record<K,V>can be transformed into an output record<K:V'>. This is a stateless record-by-record operation.The example below counts the number of token of the value string.
KTable<String, String> inputTable = builder.table("topic"); KTable<String, Integer> outputTable = inputTable.mapValue(new ValueMapper<String, Integer> { Integer apply(String value) { return value.split(" ").length; } });To query the local
KeyValueStorerepresenting outputTable above it must be obtained viaKafkaStreams#store(...): For non-local keys, a custom RPC mechanism must be implemented usingKafkaStreams.allMetadata()to query the value of the key on a parallel running instance of your Kafka Streams application.This operation preserves data co-location with respect to the key. Thus, no internal data redistribution is required if a key based operator (like a join) is applied to the result
KTable.Note that
mapValuesfor a changelog stream works differently thanrecord stream filters, becauserecordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to delete the corresponding record in the resultKTable.- Type Parameters:
VR- the value type of the resultKTable- Parameters:
mapper- aValueMapperthat computes a new output valuevalueSerde- serializer for new value typestoreSupplier- user defined state store supplier. Cannot benull.- Returns:
- a
KTablethat contains records with unmodified keys and new values (possibly of different type)
-
print
@Deprecated void print()
Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to aKStreamusingtoStream()and then useprint(Printed.toSysOut())on the result.Print the updated records of thisKTabletoSystem.out. This function will use the generated name of the parent processor node to label the key/value pairs printed to the console.The provided serde will be used to deserialize the key or value in case the type is
byte[]before callingtoString()on the deserialized object.Implementors will need to override
toString()for keys and values that are not of typeString,Integeretc. to get meaningful information.Note that
print()is not applied to the internal state store and only called for each newKTableupdated record.
-
print
@Deprecated void print(java.lang.String label)
Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to aKStreamusingtoStream()and then useprint(Printed.toSysOut().withLabel(lable))on the result.Print the updated records of thisKTabletoSystem.out. This function will use the given name to label the key/value pairs printed to the console.The provided serde will be used to deserialize the key or value in case the type is
byte[]before callingtoString()on the deserialized object.Implementors will need to override
toString()for keys and values that are not of typeString,Integeretc. to get meaningful information.Note that
print()is not applied to the internal state store and only called for each newKTableupdated record.- Parameters:
label- the name used to label the key/value pairs printed to the console
-
print
@Deprecated void print(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde)
Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to aKStreamusingtoStream()and then useprint(Printed.toSysOut().withKeyValueMapper(...)on the result.Print the updated records of thisKTabletoSystem.out. This function will use the generated name of the parent processor node to label the key/value pairs printed to the console.The provided serde will be used to deserialize the key or value in case the type is
byte[]before callingtoString()on the deserialized object.Implementors will need to override
toString()for keys and values that are not of typeString,Integeretc. to get meaningful information.Note that
print()is not applied to the internal state store and only called for each newKTableupdated record.- Parameters:
keySerde- key serde used to deserialize key if type isbyte[],valSerde- value serde used to deserialize value if type isbyte[]
-
print
@Deprecated void print(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, java.lang.String label)
Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to aKStreamusingtoStream()and then useprint(Printed.toSysOut().withLabel(label).withKeyValueMapper(...)on the result.Print the updated records of thisKTabletoSystem.out. This function will use the given name to label the key/value pairs printed to the console.The provided serde will be used to deserialize the key or value in case the type is
byte[]before callingtoString()on the deserialized object.Implementors will need to override
toString()for keys and values that are not of typeString,Integeretc. to get meaningful information.Note that
print()is not applied to the internal state store and only called for each newKTableupdated record.- Parameters:
keySerde- key serde used to deserialize key if type isbyte[],valSerde- value serde used to deserialize value if type isbyte[],label- the name used to label the key/value pairs printed to the console
-
writeAsText
@Deprecated void writeAsText(java.lang.String filePath)
Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to aKStreamusingtoStream()and then useprint(Printed.toFile(filePath)on the result.Write the updated records of thisKTableto a file at the given path. This function will use the generated name of the parent processor node to label the key/value pairs printed to the file.The default serde will be used to deserialize the key or value in case the type is
byte[]before callingtoString()on the deserialized object.Implementors will need to override
toString()for keys and values that are not of typeString,Integeretc. to get meaningful information.Note that
writeAsText()is not applied to the internal state store and only called for each newKTableupdated record.- Parameters:
filePath- name of file to write to
-
writeAsText
@Deprecated void writeAsText(java.lang.String filePath, java.lang.String label)Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to aKStreamusingtoStream()and then useprint(Printed.toFile(filePath).withLabel(label)on the result.Write the updated records of thisKTableto a file at the given path. This function will use the given name to label the key/value printed to the file.The default serde will be used to deserialize the key or value in case the type is
byte[]before callingtoString()on the deserialized object.Implementors will need to override
toString()for keys and values that are not of typeString,Integeretc. to get meaningful information.Note that
writeAsText()is not applied to the internal state store and only called for each newKTableupdated record.- Parameters:
filePath- name of file to write tolabel- the name used to label the key/value pairs printed out to the console
-
writeAsText
@Deprecated void writeAsText(java.lang.String filePath, org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde)Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to aKStreamusingtoStream()and then useprint(Printed.toFile(filePath).withKeyValueMapper(...)on the result.Write the updated records of thisKTableto a file at the given path. This function will use the generated name of the parent processor node to label the key/value pairs printed to the file.The provided serde will be used to deserialize the key or value in case the type is
byte[]before callingtoString()on the deserialized object.Implementors will need to override
toString()for keys and values that are not of typeString,Integeretc. to get meaningful information.Note that
writeAsText()is not applied to the internal state store and only called for each newKTableupdated record.- Parameters:
filePath- name of file to write tokeySerde- key serde used to deserialize key if type isbyte[],valSerde- value serde used to deserialize value if type isbyte[]
-
writeAsText
@Deprecated void writeAsText(java.lang.String filePath, java.lang.String label, org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde)Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to aKStreamusingtoStream()and then useprint(Printed.toFile(filePath).withLabel(label).withKeyValueMapper(...)on the result.Write the updated records of thisKTableto a file at the given path. This function will use the given name to label the key/value printed to the file.The default serde will be used to deserialize the key or value in case the type is
byte[]before callingtoString()on the deserialized object.Implementors will need to override
toString()for keys and values that are not of typeString,Integeretc. to get meaningful information.Note that
writeAsText()is not applied to the internal state store and only called for each newKTableupdated record.- Parameters:
filePath- name of file to write tolabel- the name used to label the key/value pairs printed to the consolekeySerde- key serde used to deserialize key if type isbyte[],valSerde- value serde used to deserialize value if type isbyte[]
-
foreach
@Deprecated void foreach(ForeachAction<? super K,? super V> action)
Deprecated.Use the Interactive Queries APIs (e.g.,KafkaStreams.store(String, QueryableStoreType)followed byReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to aKStreamusingtoStream()and then useforeach(action)on the result.Perform an action on each updated record of thisKTable. Note that this is a terminal operation that returns void.Note that
foreach()is not applied to the internal state store and only called for each newKTableupdated record.- Parameters:
action- an action to perform on each record
-
toStream
KStream<K,V> toStream()
Convert this changelog stream to aKStream.Note that this is a logical operation and only changes the "interpretation" of the stream, i.e., each record of this changelog stream is no longer treated as an updated record (cf.
KStreamvsKTable).- Returns:
- a
KStreamthat contains the same records as thisKTable
-
toStream
<KR> KStream<KR,V> toStream(KeyValueMapper<? super K,? super V,? extends KR> mapper)
Convert this changelog stream to aKStreamusing the givenKeyValueMapperto select the new key.For example, you can compute the new key as the length of the value string.
Setting a new key might result in an internal data redistribution if a key based operator (like an aggregation or join) is applied to the resultKTable<String, String> table = builder.table("topic"); KTable<Integer, String> keyedStream = table.toStream(new KeyValueMapper<String, String, Integer> { Integer apply(String key, String value) { return value.length(); } });KStream.This operation is equivalent to calling
table.toStream().selectKey(KeyValueMapper).Note that
toStream()is a logical operation and only changes the "interpretation" of the stream, i.e., each record of this changelog stream is no longer treated as an updated record (cf.KStreamvsKTable).- Type Parameters:
KR- the new key type of the result stream- Parameters:
mapper- aKeyValueMapperthat computes a new key for each record- Returns:
- a
KStreamthat contains the same records as thisKTable
-
through
@Deprecated KTable<K,V> through(java.lang.String topic, java.lang.String queryableStoreName)
Deprecated.usetoStream()followed byto(topic)andStreamsBuilder#table(topic, Materialized.as(queryableStoreName))to read back as aKTableMaterialize this changelog stream to a topic and creates a newKTablefrom the topic using default serializers and deserializers and producer'sDefaultPartitioner. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).This is equivalent to calling
#to(someTopicName)andKStreamBuilder#table(someTopicName, queryableStoreName).The resulting
KTablewill be materialized in a local state store with the given store name (cf.KStreamBuilder.table(String, String)) The store name must be a valid Kafka topic name and cannot contain characters other than ASCII alphanumerics, '.', '_' and '-'.- Parameters:
topic- the topic namequeryableStoreName- the state store name used for the resultKTable; valid characters are ASCII alphanumerics, '.', '_' and '-'. Ifnullthis is the equivalent ofthrough(String)- Returns:
- a
KTablethat contains the exact same (and potentially repartitioned) records as thisKTable
-
through
@Deprecated KTable<K,V> through(java.lang.String topic, StateStoreSupplier<KeyValueStore> storeSupplier)
Deprecated.usetoStream()followed byto(topic)and andStreamsBuilder#table(topic, Materialized.as(KeyValueBytesStoreSupplier))to read back as aKTableMaterialize this changelog stream to a topic and creates a newKTablefrom the topic using default serializers and deserializers and producer'sDefaultPartitioner. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).This is equivalent to calling
#to(someTopicName)andKStreamBuilder#table(someTopicName, queryableStoreName).The resulting
KTablewill be materialized in a local state store with the given store name (cf.KStreamBuilder.table(String, String)) The store name must be a valid Kafka topic name and cannot contain characters other than ASCII alphanumerics, '.', '_' and '-'.- Parameters:
topic- the topic namestoreSupplier- user defined state store supplier. Cannot benull.- Returns:
- a
KTablethat contains the exact same (and potentially repartitioned) records as thisKTable
-
through
@Deprecated KTable<K,V> through(java.lang.String topic)
Deprecated.Materialize this changelog stream to a topic and creates a newKTablefrom the topic using default serializers and deserializers and producer'sDefaultPartitioner. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).This is equivalent to calling
#to(someTopicName)andKStreamBuilder#table(someTopicName).The resulting
KTablewill be materialized in a local state store with an internal store name (cf.KStreamBuilder.table(String))- Parameters:
topic- the topic name- Returns:
- a
KTablethat contains the exact same (and potentially repartitioned) records as thisKTable
-
through
@Deprecated KTable<K,V> through(StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic)
Deprecated.usetoStream()followed byto(topic, Produced.streamPartitioner(partitioner))andStreamsBuilder#table(topic)to read back as aKTableMaterialize this changelog stream to a topic and creates a newKTablefrom the topic using default serializers and deserializers and a customizableStreamPartitionerto determine the distribution of records to partitions. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).This is equivalent to calling
#to(partitioner, someTopicName)andKStreamBuilder#table(someTopicName).The resulting
KTablewill be materialized in a local state store with an internal store name (cf.KStreamBuilder.table(String))- Parameters:
partitioner- the function used to determine how records are distributed among partitions of the topic, if not specified producer'sDefaultPartitionerwill be usedtopic- the topic name- Returns:
- a
KTablethat contains the exact same (and potentially repartitioned) records as thisKTable
-
through
@Deprecated KTable<K,V> through(StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic, java.lang.String queryableStoreName)
Deprecated.usetoStream()followed byto(topic, Produced.streamPartitioner(partitioner))andStreamsBuilder#table(topic, Materialized.as(queryableStoreName))to read back as aKTableMaterialize this changelog stream to a topic and creates a newKTablefrom the topic using default serializers and deserializers and a customizableStreamPartitionerto determine the distribution of records to partitions. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).This is equivalent to calling
#to(partitioner, someTopicName)andKStreamBuilder#table(someTopicName, queryableStoreName).The resulting
KTablewill be materialized in a local state store with the given store name (cf.KStreamBuilder.table(String, String))- Parameters:
partitioner- the function used to determine how records are distributed among partitions of the topic, if not specified producer'sDefaultPartitionerwill be usedtopic- the topic namequeryableStoreName- the state store name used for the resultKTable. Ifnullthis is the equivalent ofthrough(StreamPartitioner, String)- Returns:
- a
KTablethat contains the exact same (and potentially repartitioned) records as thisKTable
-
through
@Deprecated KTable<K,V> through(StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic, StateStoreSupplier<KeyValueStore> storeSupplier)
Deprecated.usetoStream()followed byto(topic, Produced.streamPartitioner(partitioner))andStreamsBuilder#table(topic, Materialized.as(KeyValueBytesStoreSupplier)to read back as aKTableMaterialize this changelog stream to a topic and creates a newKTablefrom the topic using default serializers and deserializers and a customizableStreamPartitionerto determine the distribution of records to partitions. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).This is equivalent to calling
#to(partitioner, someTopicName)andKStreamBuilder#table(someTopicName, queryableStoreName).The resulting
KTablewill be materialized in a local state store with the given store name (cf.KStreamBuilder.table(String, String))- Parameters:
partitioner- the function used to determine how records are distributed among partitions of the topic, if not specified producer'sDefaultPartitionerwill be usedtopic- the topic namestoreSupplier- user defined state store supplier. Cannot benull.- Returns:
- a
KTablethat contains the exact same (and potentially repartitioned) records as thisKTable
-
through
@Deprecated KTable<K,V> through(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, java.lang.String topic, java.lang.String queryableStoreName)
Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde))andStreamsBuilder#table(topic, Materialized.as(queryableStoreName))to read back as aKTableMaterialize this changelog stream to a topic and creates a newKTablefrom the topic. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).If
keySerdeprovides aWindowedSerializerfor the keyWindowedStreamPartitioneris used—otherwise producer'sDefaultPartitioneris used.This is equivalent to calling
#to(keySerde, valueSerde, someTopicName)andStreamsBuilder#table(someTopicName, queryableStoreName).The resulting
KTablewill be materialized in a local state store with the given store name (cf.KStreamBuilder.table(String, String))- Parameters:
keySerde- key serde used to send key-value pairs, if not specified the default key serde defined in the configuration will be usedvalSerde- value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be usedtopic- the topic namequeryableStoreName- the state store name used for the resultKTable. Ifnullthis is the equivalent ofthrough(Serde, Serde, String)- Returns:
- a
KTablethat contains the exact same (and potentially repartitioned) records as thisKTable
-
through
@Deprecated KTable<K,V> through(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, java.lang.String topic, StateStoreSupplier<KeyValueStore> storeSupplier)
Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde))andStreamsBuilder#table(topic, Materialized.as(KeyValueBytesStoreSupplier)to read back as aKTableMaterialize this changelog stream to a topic and creates a newKTablefrom the topic. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).If
keySerdeprovides aWindowedSerializerfor the keyWindowedStreamPartitioneris used—otherwise producer'sDefaultPartitioneris used.This is equivalent to calling
#to(keySerde, valueSerde, someTopicName)andKStreamBuilder#table(someTopicName, queryableStoreName).The resulting
KTablewill be materialized in a local state store with the given store name (cf.StreamsBuilder.table(String, Materialized))- Parameters:
keySerde- key serde used to send key-value pairs, if not specified the default key serde defined in the configuration will be usedvalSerde- value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be usedtopic- the topic namestoreSupplier- user defined state store supplier. Cannot benull.- Returns:
- a
KTablethat contains the exact same (and potentially repartitioned) records as thisKTable
-
through
@Deprecated KTable<K,V> through(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, java.lang.String topic)
Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde))andStreamsBuilder#table(topic)to read back as aKTableMaterialize this changelog stream to a topic and creates a newKTablefrom the topic. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).If
keySerdeprovides aWindowedSerializerfor the keyWindowedStreamPartitioneris used—otherwise producer'sDefaultPartitioneris used.This is equivalent to calling
#to(keySerde, valueSerde, someTopicName)andKStreamBuilder#table(someTopicName).The resulting
KTablewill be materialized in a local state store with an interna; store name (cf.KStreamBuilder.table(String))- Parameters:
keySerde- key serde used to send key-value pairs, if not specified the default key serde defined in the configuration will be usedvalSerde- value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be usedtopic- the topic name- Returns:
- a
KTablethat contains the exact same (and potentially repartitioned) records as thisKTable
-
through
@Deprecated KTable<K,V> through(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic, java.lang.String queryableStoreName)
Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde, partitioner))andStreamsBuilder#table(topic, Materialized.as(queryableStoreName))to read back as aKTableMaterialize this changelog stream to a topic and creates a newKTablefrom the topic using a customizableStreamPartitionerto determine the distribution of records to partitions. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).This is equivalent to calling
#to(keySerde, valueSerde, partitioner, someTopicName)andKStreamBuilder#table(someTopicName, queryableStoreName).The resulting
KTablewill be materialized in a local state store with the given store name (cf.KStreamBuilder.table(String, String))- Parameters:
keySerde- key serde used to send key-value pairs, if not specified the default key serde defined in the configuration will be usedvalSerde- value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be usedpartitioner- the function used to determine how records are distributed among partitions of the topic, if not specified andkeySerdeprovides aWindowedSerializerfor the keyWindowedStreamPartitionerwill be used—otherwiseDefaultPartitionerwill be usedtopic- the topic namequeryableStoreName- the state store name used for the resultKTable. Ifnullthis is the equivalent of {@link KTable#through(Serde, Serde, StreamPartitioner, String)()}- Returns:
- a
KTablethat contains the exact same (and potentially repartitioned) records as thisKTable
-
through
@Deprecated KTable<K,V> through(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic, StateStoreSupplier<KeyValueStore> storeSupplier)
Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde, partitioner))andStreamsBuilder#table(topic, Materialized.as(KeyValueBytesStoreSupplier))to read back as aKTableMaterialize this changelog stream to a topic and creates a newKTablefrom the topic using a customizableStreamPartitionerto determine the distribution of records to partitions. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).This is equivalent to calling
#to(keySerde, valueSerde, partitioner, someTopicName)andKStreamBuilder#table(someTopicName, queryableStoreName).The resulting
KTablewill be materialized in a local state store with the given store name (cf.KStreamBuilder.table(String, String))- Parameters:
keySerde- key serde used to send key-value pairs, if not specified the default key serde defined in the configuration will be usedvalSerde- value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be usedpartitioner- the function used to determine how records are distributed among partitions of the topic, if not specified andkeySerdeprovides aWindowedSerializerfor the keyWindowedStreamPartitionerwill be used—otherwiseDefaultPartitionerwill be usedtopic- the topic namestoreSupplier- user defined state store supplier. Cannot benull.- Returns:
- a
KTablethat contains the exact same (and potentially repartitioned) records as thisKTable
-
through
@Deprecated KTable<K,V> through(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic)
Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde, partitioner))andStreamsBuilder#table(topic)to read back as aKTableMaterialize this changelog stream to a topic and creates a newKTablefrom the topic using a customizableStreamPartitionerto determine the distribution of records to partitions. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).This is equivalent to calling
#to(keySerde, valueSerde, partitioner, someTopicName)andKStreamBuilder#table(someTopicName).The resulting
KTablewill be materialized in a local state store with an internal store name (cf.KStreamBuilder.table(String))- Parameters:
keySerde- key serde used to send key-value pairs, if not specified the default key serde defined in the configuration will be usedvalSerde- value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be usedpartitioner- the function used to determine how records are distributed among partitions of the topic, if not specified andkeySerdeprovides aWindowedSerializerfor the keyWindowedStreamPartitionerwill be used—otherwiseDefaultPartitionerwill be usedtopic- the topic name- Returns:
- a
KTablethat contains the exact same (and potentially repartitioned) records as thisKTable
-
to
@Deprecated void to(java.lang.String topic)
Deprecated.usetoStream()followed byto(topic)Materialize this changelog stream to a topic using default serializers and deserializers and producer'sDefaultPartitioner. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).- Parameters:
topic- the topic name
-
to
@Deprecated void to(StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic)
Deprecated.usetoStream()followed byto(topic, Produced.withStreamPartitioner(partitioner)Materialize this changelog stream to a topic using default serializers and deserializers and a customizableStreamPartitionerto determine the distribution of records to partitions. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).- Parameters:
partitioner- the function used to determine how records are distributed among partitions of the topic, if not specified producer'sDefaultPartitionerwill be usedtopic- the topic name
-
to
@Deprecated void to(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, java.lang.String topic)
Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde)Materialize this changelog stream to a topic. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).If
keySerdeprovides aWindowedSerializerfor the keyWindowedStreamPartitioneris used—otherwise producer'sDefaultPartitioneris used.- Parameters:
keySerde- key serde used to send key-value pairs, if not specified the default key serde defined in the configuration will be usedvalSerde- value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be usedtopic- the topic name
-
to
@Deprecated void to(org.apache.kafka.common.serialization.Serde<K> keySerde, org.apache.kafka.common.serialization.Serde<V> valSerde, StreamPartitioner<? super K,? super V> partitioner, java.lang.String topic)
Deprecated.usetoStream()followed byto(topic, Produced.with(keySerde, valSerde, partioner)Materialize this changelog stream to a topic using a customizableStreamPartitionerto determine the distribution of records to partitions. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).- Parameters:
keySerde- key serde used to send key-value pairs, if not specified the default key serde defined in the configuration will be usedvalSerde- value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be usedpartitioner- the function used to determine how records are distributed among partitions of the topic, if not specified andkeySerdeprovides aWindowedSerializerfor the keyWindowedStreamPartitionerwill be used—otherwiseDefaultPartitionerwill be usedtopic- the topic name
-
groupBy
<KR,VR> KGroupedTable<KR,VR> groupBy(KeyValueMapper<? super K,? super V,KeyValue<KR,VR>> selector)
Re-groups the records of thisKTableusing the providedKeyValueMapperand default serializers and deserializers. EachKeyValuepair of thisKTableis mapped to a newKeyValuepair by applying the providedKeyValueMapper. Re-grouping aKTableis required before an aggregation operator can be applied to the data (cf.KGroupedTable). TheKeyValueMapperselects a new key and value (with should both have unmodified type). If the new record key isnullthe record will not be included in the resultingKGroupedTableBecause a new key is selected, an internal repartitioning topic will be created in Kafka. This topic will be named "${applicationId}-XXX-repartition", where "applicationId" is user-specified in
StreamsConfigvia parameterAPPLICATION_ID_CONFIG, "XXX" is an internally generated name, and "-repartition" is a fixed suffix. You can retrieve all generated internal topic names viaKafkaStreams.toString().All data of this
KTablewill be redistributed through the repartitioning topic by writing all update records to and rereading all updated records from it, such that the resultingKGroupedTableis partitioned on the new key.If the key or value type is changed, it is recommended to use
groupBy(KeyValueMapper, Serialized)instead.- Type Parameters:
KR- the key type of the resultKGroupedTableVR- the value type of the resultKGroupedTable- Parameters:
selector- aKeyValueMapperthat computes a new grouping key and value to be aggregated- Returns:
- a
KGroupedTablethat contains the re-grouped records of the originalKTable
-
groupBy
<KR,VR> KGroupedTable<KR,VR> groupBy(KeyValueMapper<? super K,? super V,KeyValue<KR,VR>> selector, Serialized<KR,VR> serialized)
Re-groups the records of thisKTableusing the providedKeyValueMapperandSerdes as specified bySerialized. EachKeyValuepair of thisKTableis mapped to a newKeyValuepair by applying the providedKeyValueMapper. Re-grouping aKTableis required before an aggregation operator can be applied to the data (cf.KGroupedTable). TheKeyValueMapperselects a new key and value (with should both have unmodified type). If the new record key isnullthe record will not be included in the resultingKGroupedTableBecause a new key is selected, an internal repartitioning topic will be created in Kafka. This topic will be named "${applicationId}-XXX-repartition", where "applicationId" is user-specified in
StreamsConfigvia parameterAPPLICATION_ID_CONFIG, "XXX" is an internally generated name, and "-repartition" is a fixed suffix. You can retrieve all generated internal topic names viaKafkaStreams.toString().All data of this
KTablewill be redistributed through the repartitioning topic by writing all update records to and rereading all updated records from it, such that the resultingKGroupedTableis partitioned on the new key.- Type Parameters:
KR- the key type of the resultKGroupedTableVR- the value type of the resultKGroupedTable- Parameters:
selector- aKeyValueMapperthat computes a new grouping key and value to be aggregatedserialized- theSerializedinstance used to specifySerdes- Returns:
- a
KGroupedTablethat contains the re-grouped records of the originalKTable
-
groupBy
@Deprecated <KR,VR> KGroupedTable<KR,VR> groupBy(KeyValueMapper<? super K,? super V,KeyValue<KR,VR>> selector, org.apache.kafka.common.serialization.Serde<KR> keySerde, org.apache.kafka.common.serialization.Serde<VR> valueSerde)
Deprecated.Re-groups the records of thisKTableusing the providedKeyValueMapper. EachKeyValuepair of thisKTableis mapped to a newKeyValuepair by applying the providedKeyValueMapper. Re-grouping aKTableis required before an aggregation operator can be applied to the data (cf.KGroupedTable). TheKeyValueMapperselects a new key and value (both with potentially different type). If the new record key isnullthe record will not be included in the resultingKGroupedTableBecause a new key is selected, an internal repartitioning topic will be created in Kafka. This topic will be named "${applicationId}-XXX-repartition", where "applicationId" is user-specified in
StreamsConfigvia parameterAPPLICATION_ID_CONFIG, "XXX" is an internally generated name, and "-repartition" is a fixed suffix. You can retrieve all generated internal topic names viaKafkaStreams.toString().All data of this
KTablewill be redistributed through the repartitioning topic by writing all update records to and rereading all updated records from it, such that the resultingKGroupedTableis partitioned on the new key.- Type Parameters:
KR- the key type of the resultKGroupedTableVR- the value type of the resultKGroupedTable- Parameters:
selector- aKeyValueMapperthat computes a new grouping key and value to be aggregatedkeySerde- key serdes for materializing this stream, if not specified the default serdes defined in the configs will be usedvalueSerde- value serdes for materializing this stream, if not specified the default serdes defined in the configs will be used- Returns:
- a
KGroupedTablethat contains the re-grouped records of the originalKTable
-
join
<VO,VR> KTable<K,VR> join(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner)
Join records of thisKTablewith anotherKTable's records using non-windowed inner equi join. The join is a primary key join with join attributethisKTable.key == otherKTable.key. The result is an ever updatingKTablethat represents the current (i.e., processing time) result of the join.The join is computed by (1) updating the internal state of one
KTableand (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the otherKTable. This happens in a symmetric way, i.e., for each update of eitherthisor theotherinputKTablethe result gets updated.For each
KTablerecord that finds a corresponding record in the otherKTablethe providedValueJoinerwill be called to compute a value (with arbitrary type) for the result record. The key of the result record is the same as for both joining input records.Note that
recordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly to delete a record in the resultKTableif required (i.e., if there is anything to be deleted).Input records with
nullkey will be dropped and no join computation is performed.Example:
Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.thisKTable thisState otherKTable otherState result updated record <K1:A> <K1:A> <K1:A> <K1:b> <K1:b> <K1:ValueJoiner(A,b)> <K1:C> <K1:C> <K1:b> <K1:ValueJoiner(C,b)> <K1:C> <K1:null> <K1:null> - Type Parameters:
VO- the value type of the otherKTableVR- the value type of the resultKTable- Parameters:
other- the otherKTableto be joined with thisKTablejoiner- aValueJoinerthat computes the join result for a pair of matching records- Returns:
- a
KTablethat contains join-records for each key and values computed by the givenValueJoiner, one for each matched record-pair with the same key - See Also:
leftJoin(KTable, ValueJoiner),outerJoin(KTable, ValueJoiner)
-
join
<VO,VR> KTable<K,VR> join(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, Materialized<K,VR,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Join records of thisKTablewith anotherKTable's records using non-windowed inner equi join. The join is a primary key join with join attributethisKTable.key == otherKTable.key. The result is an ever updatingKTablethat represents the current (i.e., processing time) result of the join.The join is computed by (1) updating the internal state of one
KTableand (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the otherKTable. This happens in a symmetric way, i.e., for each update of eitherthisor theotherinputKTablethe result gets updated.For each
KTablerecord that finds a corresponding record in the otherKTablethe providedValueJoinerwill be called to compute a value (with arbitrary type) for the result record. The key of the result record is the same as for both joining input records.Note that
recordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly to delete a record in the resultKTableif required (i.e., if there is anything to be deleted).Input records with
nullkey will be dropped and no join computation is performed.Example:
Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.thisKTable thisState otherKTable otherState result updated record <K1:A> <K1:A> <K1:A> <K1:b> <K1:b> <K1:ValueJoiner(A,b)> <K1:C> <K1:C> <K1:b> <K1:ValueJoiner(C,b)> <K1:C> <K1:null> <K1:null> - Type Parameters:
VO- the value type of the otherKTableVR- the value type of the resultKTable- Parameters:
other- the otherKTableto be joined with thisKTablejoiner- aValueJoinerthat computes the join result for a pair of matching recordsmaterialized- an instance ofMaterializedused to describe how the state store should be materialized. Cannot benull- Returns:
- a
KTablethat contains join-records for each key and values computed by the givenValueJoiner, one for each matched record-pair with the same key - See Also:
leftJoin(KTable, ValueJoiner, Materialized),outerJoin(KTable, ValueJoiner, Materialized)
-
join
@Deprecated <VO,VR> KTable<K,VR> join(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, org.apache.kafka.common.serialization.Serde<VR> joinSerde, java.lang.String queryableStoreName)
Join records of thisKTablewith anotherKTable's records using non-windowed inner equi join. The join is a primary key join with join attributethisKTable.key == otherKTable.key. The result is an ever updatingKTablethat represents the current (i.e., processing time) result of the join.The join is computed by (1) updating the internal state of one
KTableand (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the otherKTable. This happens in a symmetric way, i.e., for each update of eitherthisor theotherinputKTablethe result gets updated.For each
KTablerecord that finds a corresponding record in the otherKTablethe providedValueJoinerwill be called to compute a value (with arbitrary type) for the result record. The key of the result record is the same as for both joining input records.Note that
recordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly to delete a record in the resultKTableif required (i.e., if there is anything to be deleted).Input records with
nullkey will be dropped and no join computation is performed.Example:
Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.thisKTable thisState otherKTable otherState result updated record <K1:A> <K1:A> <K1:A> <K1:b> <K1:b> <K1:ValueJoiner(A,b)> <K1:C> <K1:C> <K1:b> <K1:ValueJoiner(C,b)> <K1:C> <K1:null> <K1:null> - Type Parameters:
VO- the value type of the otherKTableVR- the value type of the resultKTable- Parameters:
other- the otherKTableto be joined with thisKTablejoiner- aValueJoinerthat computes the join result for a pair of matching recordsjoinSerde- serializer for join result value typequeryableStoreName- a user-provided name of the underlyingKTablethat can be used to subsequently query the operation results; valid characters are ASCII alphanumerics, '.', '_' and '-'. Ifnullthen the results cannot be queried (i.e., that would be equivalent to callingjoin(KTable, ValueJoiner).- Returns:
- a
KTablethat contains join-records for each key and values computed by the givenValueJoiner, one for each matched record-pair with the same key - See Also:
leftJoin(KTable, ValueJoiner, Materialized),outerJoin(KTable, ValueJoiner, Materialized)
-
join
@Deprecated <VO,VR> KTable<K,VR> join(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, StateStoreSupplier<KeyValueStore> storeSupplier)
Deprecated.Join records of thisKTablewith anotherKTable's records using non-windowed inner equi join. The join is a primary key join with join attributethisKTable.key == otherKTable.key. The result is an ever updatingKTablethat represents the current (i.e., processing time) result of the join.The join is computed by (1) updating the internal state of one
KTableand (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the otherKTable. This happens in a symmetric way, i.e., for each update of eitherthisor theotherinputKTablethe result gets updated.For each
KTablerecord that finds a corresponding record in the otherKTablethe providedValueJoinerwill be called to compute a value (with arbitrary type) for the result record. The key of the result record is the same as for both joining input records.Note that
recordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly to delete a record in the resultKTableif required (i.e., if there is anything to be deleted).Input records with
nullkey will be dropped and no join computation is performed.Example:
Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.thisKTable thisState otherKTable otherState result updated record <K1:A> <K1:A> <K1:A> <K1:b> <K1:b> <K1:ValueJoiner(A,b)> <K1:C> <K1:C> <K1:b> <K1:ValueJoiner(C,b)> <K1:C> <K1:null> <K1:null> - Type Parameters:
VO- the value type of the otherKTableVR- the value type of the resultKTable- Parameters:
other- the otherKTableto be joined with thisKTablejoiner- aValueJoinerthat computes the join result for a pair of matching recordsstoreSupplier- user defined state store supplier. Cannot benull.- Returns:
- a
KTablethat contains join-records for each key and values computed by the givenValueJoiner, one for each matched record-pair with the same key - See Also:
leftJoin(KTable, ValueJoiner, Materialized),outerJoin(KTable, ValueJoiner, Materialized)
-
leftJoin
<VO,VR> KTable<K,VR> leftJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner)
Join records of thisKTable(left input) with anotherKTable's (right input) records using non-windowed left equi join. The join is a primary key join with join attributethisKTable.key == otherKTable.key. In contrast toinner-join, all records from leftKTablewill produce an output record (cf. below). The result is an ever updatingKTablethat represents the current (i.e., processing time) result of the join.The join is computed by (1) updating the internal state of one
KTableand (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the otherKTable. This happens in a symmetric way, i.e., for each update of eitherthisor theotherinputKTablethe result gets updated.For each
KTablerecord that finds a corresponding record in the otherKTable's state the providedValueJoinerwill be called to compute a value (with arbitrary type) for the result record. Additionally, for each record of leftKTablethat does not find a corresponding record in the rightKTable's state the providedValueJoinerwill be called withrightValue = nullto compute a value (with arbitrary type) for the result record. The key of the result record is the same as for both joining input records.Note that
recordswithnullvalues (so-called tombstone records) have delete semantics. For example, for left input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly to delete a record in the resultKTableif required (i.e., if there is anything to be deleted).Input records with
nullkey will be dropped and no join computation is performed.Example:
Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.thisKTable thisState otherKTable otherState result updated record <K1:A> <K1:A> <K1:ValueJoiner(A,null)> <K1:A> <K1:b> <K1:b> <K1:ValueJoiner(A,b)> <K1:null> <K1:b> <K1:null> <K1:null> - Type Parameters:
VO- the value type of the otherKTableVR- the value type of the resultKTable- Parameters:
other- the otherKTableto be joined with thisKTablejoiner- aValueJoinerthat computes the join result for a pair of matching records- Returns:
- a
KTablethat contains join-records for each key and values computed by the givenValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of leftKTable - See Also:
join(KTable, ValueJoiner),outerJoin(KTable, ValueJoiner)
-
leftJoin
<VO,VR> KTable<K,VR> leftJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, Materialized<K,VR,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Join records of thisKTable(left input) with anotherKTable's (right input) records using non-windowed left equi join. The join is a primary key join with join attributethisKTable.key == otherKTable.key. In contrast toinner-join, all records from leftKTablewill produce an output record (cf. below). The result is an ever updatingKTablethat represents the current (i.e., processing time) result of the join.The join is computed by (1) updating the internal state of one
KTableand (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the otherKTable. This happens in a symmetric way, i.e., for each update of eitherthisor theotherinputKTablethe result gets updated.For each
KTablerecord that finds a corresponding record in the otherKTable's state the providedValueJoinerwill be called to compute a value (with arbitrary type) for the result record. Additionally, for each record of leftKTablethat does not find a corresponding record in the rightKTable's state the providedValueJoinerwill be called withrightValue = nullto compute a value (with arbitrary type) for the result record. The key of the result record is the same as for both joining input records.Note that
recordswithnullvalues (so-called tombstone records) have delete semantics. For example, for left input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly to delete a record in the resultKTableif required (i.e., if there is anything to be deleted).Input records with
nullkey will be dropped and no join computation is performed.Example:
Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.thisKTable thisState otherKTable otherState result updated record <K1:A> <K1:A> <K1:ValueJoiner(A,null)> <K1:A> <K1:b> <K1:b> <K1:ValueJoiner(A,b)> <K1:null> <K1:b> <K1:null> <K1:null> - Type Parameters:
VO- the value type of the otherKTableVR- the value type of the resultKTable- Parameters:
other- the otherKTableto be joined with thisKTablejoiner- aValueJoinerthat computes the join result for a pair of matching recordsmaterialized- an instance ofMaterializedused to describe how the state store should be materialized. Cannot benull- Returns:
- a
KTablethat contains join-records for each key and values computed by the givenValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of leftKTable - See Also:
join(KTable, ValueJoiner, Materialized),outerJoin(KTable, ValueJoiner, Materialized)
-
leftJoin
@Deprecated <VO,VR> KTable<K,VR> leftJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, org.apache.kafka.common.serialization.Serde<VR> joinSerde, java.lang.String queryableStoreName)
Deprecated.Join records of thisKTable(left input) with anotherKTable's (right input) records using non-windowed left equi join. The join is a primary key join with join attributethisKTable.key == otherKTable.key. In contrast toinner-join, all records from leftKTablewill produce an output record (cf. below). The result is an ever updatingKTablethat represents the current (i.e., processing time) result of the join.The join is computed by (1) updating the internal state of one
KTableand (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the otherKTable. This happens in a symmetric way, i.e., for each update of eitherthisor theotherinputKTablethe result gets updated.For each
KTablerecord that finds a corresponding record in the otherKTable's state the providedValueJoinerwill be called to compute a value (with arbitrary type) for the result record. Additionally, for each record of leftKTablethat does not find a corresponding record in the rightKTable's state the providedValueJoinerwill be called withrightValue = nullto compute a value (with arbitrary type) for the result record. The key of the result record is the same as for both joining input records.Note that
recordswithnullvalues (so-called tombstone records) have delete semantics. For example, for left input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly to delete a record in the resultKTableif required (i.e., if there is anything to be deleted).Input records with
nullkey will be dropped and no join computation is performed.Example:
Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.thisKTable thisState otherKTable otherState result updated record <K1:A> <K1:A> <K1:ValueJoiner(A,null)> <K1:A> <K1:b> <K1:b> <K1:ValueJoiner(A,b)> <K1:null> <K1:b> <K1:null> <K1:null> - Type Parameters:
VO- the value type of the otherKTableVR- the value type of the resultKTable- Parameters:
other- the otherKTableto be joined with thisKTablejoiner- aValueJoinerthat computes the join result for a pair of matching recordsjoinSerde- serializer for join result value typequeryableStoreName- a user-provided name of the underlyingKTablethat can be used to subsequently query the operation results; valid characters are ASCII alphanumerics, '.', '_' and '-'. Ifnullthen the results cannot be queried (i.e., that would be equivalent to callingleftJoin(KTable, ValueJoiner).- Returns:
- a
KTablethat contains join-records for each key and values computed by the givenValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of leftKTable - See Also:
join(KTable, ValueJoiner, Materialized),outerJoin(KTable, ValueJoiner, Materialized)
-
leftJoin
@Deprecated <VO,VR> KTable<K,VR> leftJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, StateStoreSupplier<KeyValueStore> storeSupplier)
Join records of thisKTable(left input) with anotherKTable's (right input) records using non-windowed left equi join. The join is a primary key join with join attributethisKTable.key == otherKTable.key. In contrast toinner-join, all records from leftKTablewill produce an output record (cf. below). The result is an ever updatingKTablethat represents the current (i.e., processing time) result of the join.The join is computed by (1) updating the internal state of one
KTableand (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the otherKTable. This happens in a symmetric way, i.e., for each update of eitherthisor theotherinputKTablethe result gets updated.For each
KTablerecord that finds a corresponding record in the otherKTable's state the providedValueJoinerwill be called to compute a value (with arbitrary type) for the result record. Additionally, for each record of leftKTablethat does not find a corresponding record in the rightKTable's state the providedValueJoinerwill be called withrightValue = nullto compute a value (with arbitrary type) for the result record. The key of the result record is the same as for both joining input records.Note that
recordswithnullvalues (so-called tombstone records) have delete semantics. For example, for left input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly to delete a record in the resultKTableif required (i.e., if there is anything to be deleted).Input records with
nullkey will be dropped and no join computation is performed.Example:
Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.thisKTable thisState otherKTable otherState result updated record <K1:A> <K1:A> <K1:ValueJoiner(A,null)> <K1:A> <K1:b> <K1:b> <K1:ValueJoiner(A,b)> <K1:null> <K1:b> <K1:null> <K1:null> - Type Parameters:
VO- the value type of the otherKTableVR- the value type of the resultKTable- Parameters:
other- the otherKTableto be joined with thisKTablejoiner- aValueJoinerthat computes the join result for a pair of matching recordsstoreSupplier- user defined state store supplier. Cannot benull.- Returns:
- a
KTablethat contains join-records for each key and values computed by the givenValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of leftKTable - See Also:
join(KTable, ValueJoiner, Materialized),outerJoin(KTable, ValueJoiner, Materialized)
-
outerJoin
<VO,VR> KTable<K,VR> outerJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner)
Join records of thisKTable(left input) with anotherKTable's (right input) records using non-windowed outer equi join. The join is a primary key join with join attributethisKTable.key == otherKTable.key. In contrast toinner-joinorleft-join, all records from both inputKTables will produce an output record (cf. below). The result is an ever updatingKTablethat represents the current (i.e., processing time) result of the join.The join is computed by (1) updating the internal state of one
KTableand (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the otherKTable. This happens in a symmetric way, i.e., for each update of eitherthisor theotherinputKTablethe result gets updated.For each
KTablerecord that finds a corresponding record in the otherKTable's state the providedValueJoinerwill be called to compute a value (with arbitrary type) for the result record. Additionally, for each record that does not find a corresponding record in the corresponding otherKTable's state the providedValueJoinerwill be called withnullvalue for the corresponding other value to compute a value (with arbitrary type) for the result record. The key of the result record is the same as for both joining input records.Note that
recordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly to delete a record in the resultKTableif required (i.e., if there is anything to be deleted).Input records with
nullkey will be dropped and no join computation is performed.Example:
Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.thisKTable thisState otherKTable otherState result updated record <K1:A> <K1:A> <K1:ValueJoiner(A,null)> <K1:A> <K1:b> <K1:b> <K1:ValueJoiner(A,b)> <K1:null> <K1:b> <K1:ValueJoiner(null,b)> <K1:null> <K1:null> - Type Parameters:
VO- the value type of the otherKTableVR- the value type of the resultKTable- Parameters:
other- the otherKTableto be joined with thisKTablejoiner- aValueJoinerthat computes the join result for a pair of matching records- Returns:
- a
KTablethat contains join-records for each key and values computed by the givenValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of bothKTables - See Also:
join(KTable, ValueJoiner),leftJoin(KTable, ValueJoiner)
-
outerJoin
<VO,VR> KTable<K,VR> outerJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, Materialized<K,VR,KeyValueStore<org.apache.kafka.common.utils.Bytes,byte[]>> materialized)
Join records of thisKTable(left input) with anotherKTable's (right input) records using non-windowed outer equi join. The join is a primary key join with join attributethisKTable.key == otherKTable.key. In contrast toinner-joinorleft-join, all records from both inputKTables will produce an output record (cf. below). The result is an ever updatingKTablethat represents the current (i.e., processing time) result of the join.The join is computed by (1) updating the internal state of one
KTableand (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the otherKTable. This happens in a symmetric way, i.e., for each update of eitherthisor theotherinputKTablethe result gets updated.For each
KTablerecord that finds a corresponding record in the otherKTable's state the providedValueJoinerwill be called to compute a value (with arbitrary type) for the result record. Additionally, for each record that does not find a corresponding record in the corresponding otherKTable's state the providedValueJoinerwill be called withnullvalue for the corresponding other value to compute a value (with arbitrary type) for the result record. The key of the result record is the same as for both joining input records.Note that
recordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly to delete a record in the resultKTableif required (i.e., if there is anything to be deleted).Input records with
nullkey will be dropped and no join computation is performed.Example:
Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.thisKTable thisState otherKTable otherState result updated record <K1:A> <K1:A> <K1:ValueJoiner(A,null)> <K1:A> <K1:b> <K1:b> <K1:ValueJoiner(A,b)> <K1:null> <K1:b> <K1:ValueJoiner(null,b)> <K1:null> <K1:null> - Type Parameters:
VO- the value type of the otherKTableVR- the value type of the resultKTable- Parameters:
other- the otherKTableto be joined with thisKTablejoiner- aValueJoinerthat computes the join result for a pair of matching recordsmaterialized- an instance ofMaterializedused to describe how the state store should be materialized. Cannot benull- Returns:
- a
KTablethat contains join-records for each key and values computed by the givenValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of bothKTables - See Also:
join(KTable, ValueJoiner),leftJoin(KTable, ValueJoiner)
-
outerJoin
@Deprecated <VO,VR> KTable<K,VR> outerJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, org.apache.kafka.common.serialization.Serde<VR> joinSerde, java.lang.String queryableStoreName)
Deprecated.Join records of thisKTable(left input) with anotherKTable's (right input) records using non-windowed outer equi join. The join is a primary key join with join attributethisKTable.key == otherKTable.key. In contrast toinner-joinorleft-join, all records from both inputKTables will produce an output record (cf. below). The result is an ever updatingKTablethat represents the current (i.e., processing time) result of the join.The join is computed by (1) updating the internal state of one
KTableand (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the otherKTable. This happens in a symmetric way, i.e., for each update of eitherthisor theotherinputKTablethe result gets updated.For each
KTablerecord that finds a corresponding record in the otherKTable's state the providedValueJoinerwill be called to compute a value (with arbitrary type) for the result record. Additionally, for each record that does not find a corresponding record in the corresponding otherKTable's state the providedValueJoinerwill be called withnullvalue for the corresponding other value to compute a value (with arbitrary type) for the result record. The key of the result record is the same as for both joining input records.Note that
recordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly to delete a record in the resultKTableif required (i.e., if there is anything to be deleted).Input records with
nullkey will be dropped and no join computation is performed.Example:
Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.thisKTable thisState otherKTable otherState result updated record <K1:A> <K1:A> <K1:ValueJoiner(A,null)> <K1:A> <K1:b> <K1:b> <K1:ValueJoiner(A,b)> <K1:null> <K1:b> <K1:ValueJoiner(null,b)> <K1:null> <K1:null> - Type Parameters:
VO- the value type of the otherKTableVR- the value type of the resultKTable- Parameters:
other- the otherKTableto be joined with thisKTablejoiner- aValueJoinerthat computes the join result for a pair of matching recordsjoinSerde- serializer for join result value typequeryableStoreName- a user-provided name of the underlyingKTablethat can be used to subsequently query the operation results; valid characters are ASCII alphanumerics, '.', '_' and '-'. Ifnullthen the results cannot be queried (i.e., that would be equivalent to callingouterJoin(KTable, ValueJoiner).- Returns:
- a
KTablethat contains join-records for each key and values computed by the givenValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of bothKTables - See Also:
join(KTable, ValueJoiner, Materialized),leftJoin(KTable, ValueJoiner, Materialized)
-
outerJoin
@Deprecated <VO,VR> KTable<K,VR> outerJoin(KTable<K,VO> other, ValueJoiner<? super V,? super VO,? extends VR> joiner, StateStoreSupplier<KeyValueStore> storeSupplier)
Join records of thisKTable(left input) with anotherKTable's (right input) records using non-windowed outer equi join. The join is a primary key join with join attributethisKTable.key == otherKTable.key. In contrast toinner-joinorleft-join, all records from both inputKTables will produce an output record (cf. below). The result is an ever updatingKTablethat represents the current (i.e., processing time) result of the join.The join is computed by (1) updating the internal state of one
KTableand (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the otherKTable. This happens in a symmetric way, i.e., for each update of eitherthisor theotherinputKTablethe result gets updated.For each
KTablerecord that finds a corresponding record in the otherKTable's state the providedValueJoinerwill be called to compute a value (with arbitrary type) for the result record. Additionally, for each record that does not find a corresponding record in the corresponding otherKTable's state the providedValueJoinerwill be called withnullvalue for the corresponding other value to compute a value (with arbitrary type) for the result record. The key of the result record is the same as for both joining input records.Note that
recordswithnullvalues (so-called tombstone records) have delete semantics. Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly to delete a record in the resultKTableif required (i.e., if there is anything to be deleted).Input records with
nullkey will be dropped and no join computation is performed.Example:
Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.thisKTable thisState otherKTable otherState result updated record <K1:A> <K1:A> <K1:ValueJoiner(A,null)> <K1:A> <K1:b> <K1:b> <K1:ValueJoiner(A,b)> <K1:null> <K1:b> <K1:ValueJoiner(null,b)> <K1:null> <K1:null> - Type Parameters:
VO- the value type of the otherKTableVR- the value type of the resultKTable- Parameters:
other- the otherKTableto be joined with thisKTablejoiner- aValueJoinerthat computes the join result for a pair of matching recordsstoreSupplier- user defined state store supplier. Cannot benull.- Returns:
- a
KTablethat contains join-records for each key and values computed by the givenValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of bothKTables - See Also:
join(KTable, ValueJoiner),leftJoin(KTable, ValueJoiner)
-
queryableStoreName
java.lang.String queryableStoreName()
Get the name of the local state store used that can be used to query thisKTable.- Returns:
- the underlying state store name, or
nullif thisKTablecannot be queried.
-
-