Interface KTable<K,​V>

  • Type Parameters:
    K - Type of primary keys
    V - Type of value changes

    @Evolving
    public interface KTable<K,​V>
    KTable is 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 KTable is either defined from a single Kafka topic that is consumed message by message or the result of a KTable transformation. An aggregation of a KStream also yields a KTable.

    A KTable can be transformed record by record, joined with another KTable or KStream, or can be re-partitioned and aggregated into a new KTable.

    Some KTables have an internal state (a ReadOnlyKeyValueStore) 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 Detail

      • filter

        KTable<K,​V> filter​(Predicate<? super K,​? super V> predicate)
        Create a new KTable that consists of all records of this KTable which satisfy the given predicate. All records that do not satisfy the predicate are dropped. For each KTable update, the filter is evaluated based on the current update record and then an update record is produced for the result KTable. This is a stateless record-by-record operation.

        Note that filter for a changelog stream works differently than record stream filters, because records with null values (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 filter Predicate that is applied to each record
        Returns:
        a KTable that 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 new KTable that consists of all records of this KTable which satisfy the given predicate. All records that do not satisfy the predicate are dropped. For each KTable update, the filter is evaluated based on the current update record and then an update record is produced for the result KTable. This is a stateless record-by-record operation.

        Note that filter for a changelog stream works differently than record stream filters, because records with null values (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 KeyValueStore it must be obtained via KafkaStreams#store(...):

        
         KafkaStreams 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)
         
        For non-local keys, a custom RPC mechanism must be implemented using 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 by Materialized.as(String) or Materialized.as(KeyValueBytesStoreSupplier).

        Parameters:
        predicate - a filter Predicate that is applied to each record
        materialized - a Materialized that describes how the StateStore for the resulting KTable should be materialized. Cannot be null
        Returns:
        a KTable that 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)
        Create a new KTable that consists of all records of this KTable which satisfy the given predicate. All records that do not satisfy the predicate are dropped. For each KTable update the filter is evaluated on the updated record to produce an updated record for the result KTable. This is a stateless record-by-record operation.

        Note that filter for a changelog stream works differently than record stream filters, because records with null values (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 KeyValueStore it must be obtained via KafkaStreams#store(...):

        
         KafkaStreams 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)
         
        For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to query the value of the key on a parallel running instance of your Kafka Streams application.

        Parameters:
        predicate - a filter Predicate that is applied to each record
        queryableStoreName - a user-provided name of the underlying KTable that can be used to subsequently query the operation results; valid characters are ASCII alphanumerics, '.', '_' and '-'. If null then the results cannot be queried (i.e., that would be equivalent to calling filter(Predicate).
        Returns:
        a KTable that 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)
        Create a new KTable that consists of all records of this KTable which satisfy the given predicate. All records that do not satisfy the predicate are dropped. For each KTable update the filter is evaluated on the updated record to produce an updated record for the result KTable. This is a stateless record-by-record operation.

        Note that filter for a changelog stream works differently than record stream filters, because records with null values (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 KeyValueStore it must be obtained via KafkaStreams#store(...):

        
         KafkaStreams 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)
         
        For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to query the value of the key on a parallel running instance of your Kafka Streams application.

        Parameters:
        predicate - a filter Predicate that is applied to each record
        storeSupplier - user defined state store supplier. Cannot be null.
        Returns:
        a KTable that 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 new KTable that consists all records of this KTable which do not satisfy the given predicate. All records that do satisfy the predicate are dropped. For each KTable update, the filter is evaluated based on the current update record and then an update record is produced for the result KTable. This is a stateless record-by-record operation.

        Note that filterNot for a changelog stream works differently than record stream filters, because records with null values (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 filter Predicate that is applied to each record
        Returns:
        a KTable that 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 new KTable that consists all records of this KTable which do not satisfy the given predicate. All records that do satisfy the predicate are dropped. For each KTable update, the filter is evaluated based on the current update record and then an update record is produced for the result KTable. This is a stateless record-by-record operation.

        Note that filterNot for a changelog stream works differently than record stream filters, because records with null values (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 KeyValueStore it must be obtained via KafkaStreams#store(...):

        
         KafkaStreams 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)
         
        For non-local keys, a custom RPC mechanism must be implemented using 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 by Materialized.as(String) or Materialized.as(KeyValueBytesStoreSupplier).

        Parameters:
        predicate - a filter Predicate that is applied to each record
        materialized - a Materialized that describes how the StateStore for the resulting KTable should be materialized. Cannot be null
        Returns:
        a KTable that 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)
        Create a new KTable that consists all records of this KTable which do not satisfy the given predicate. All records that do satisfy the predicate are dropped. For each KTable update the filter is evaluated on the updated record to produce an updated record for the result KTable. This is a stateless record-by-record operation.

        Note that filterNot for a changelog stream works differently than record stream filters, because records with null values (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 KeyValueStore it must be obtained via KafkaStreams#store(...):

        
         KafkaStreams 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)
         
        For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to query the value of the key on a parallel running instance of your Kafka Streams application.

        Parameters:
        predicate - a filter Predicate that is applied to each record
        storeSupplier - user defined state store supplier. Cannot be null.
        Returns:
        a KTable that 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)
        Create a new KTable that consists all records of this KTable which do not satisfy the given predicate. All records that do satisfy the predicate are dropped. For each KTable update the filter is evaluated on the updated record to produce an updated record for the result KTable. This is a stateless record-by-record operation.

        Note that filterNot for a changelog stream works differently than record stream filters, because records with null values (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 KeyValueStore it must be obtained via KafkaStreams#store(...):

        
         KafkaStreams 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)
         
        For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.allMetadata() to query the value of the key on a parallel running instance of your Kafka Streams application.

        Parameters:
        predicate - a filter Predicate that is applied to each record
        queryableStoreName - a user-provided name of the underlying KTable that can be used to subsequently query the operation results; valid characters are ASCII alphanumerics, '.', '_' and '-'. If null then the results cannot be queried (i.e., that would be equivalent to calling filterNot(Predicate).
        Returns:
        a KTable that 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 new KTable by transforming the value of each record in this KTable into a new value (with possible new type) in the new KTable. For each KTable update the provided ValueMapper is applied to the value of the updated record and computes a new value for it, resulting in an updated record for the result KTable. 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 mapValues for a changelog stream works differently than record stream filters, because records with null values (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 result KTable.

        Type Parameters:
        VR - the value type of the result KTable
        Parameters:
        mapper - a ValueMapper that computes a new output value
        Returns:
        a KTable that 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 new KTable by transforming the value of each record in this KTable into a new value (with possible new type) in the new KTable. For each KTable update the provided ValueMapperWithKey is applied to the value of the update record and computes a new value for it, resulting in an updated record for the result KTable. 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 mapValues for a changelog stream works differently than record stream filters, because records with null values (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 result KTable.

        Type Parameters:
        VR - the value type of the result KTable
        Parameters:
        mapper - a ValueMapperWithKey that computes a new output value
        Returns:
        a KTable that 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 new KTable by transforming the value of each record in this KTable into a new value (with possible new type) in the new KTable. For each KTable update the provided ValueMapper is applied to the value of the updated record and computes a new value for it, resulting in an updated record for the result KTable. 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 KeyValueStore representing outputTable above it must be obtained via KafkaStreams#store(...): For non-local keys, a custom RPC mechanism must be implemented using 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 by Materialized.as(String) or Materialized.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 mapValues for a changelog stream works differently than record stream filters, because records with null values (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 result KTable.

        Type Parameters:
        VR - the value type of the result KTable
        Parameters:
        mapper - a ValueMapper that computes a new output value
        materialized - a Materialized that describes how the StateStore for the resulting KTable should be materialized. Cannot be null
        Returns:
        a KTable that 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 new KTable by transforming the value of each record in this KTable into a new value (with possible new type) in the new KTable. For each KTable update the provided ValueMapperWithKey is applied to the value of the update record and computes a new value for it, resulting in an updated record for the result KTable. 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 KeyValueStore representing outputTable above it must be obtained via KafkaStreams#store(...): For non-local keys, a custom RPC mechanism must be implemented using 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 by Materialized.as(String) or Materialized.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 mapValues for a changelog stream works differently than record stream filters, because records with null values (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 result KTable.

        Type Parameters:
        VR - the value type of the result KTable
        Parameters:
        mapper - a ValueMapperWithKey that computes a new output value
        materialized - a Materialized that describes how the StateStore for the resulting KTable should be materialized. Cannot be null
        Returns:
        a KTable that 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 new KTable by transforming the value of each record in this KTable into a new value (with possible new type) in the new KTable. For each KTable update the provided ValueMapper is applied to the value of the updated record and computes a new value for it, resulting in an updated record for the result KTable. 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 KeyValueStore representing outputTable above it must be obtained via KafkaStreams#store(...): For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.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 mapValues for a changelog stream works differently than record stream filters, because records with null values (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 result KTable.

        Type Parameters:
        VR - the value type of the result KTable
        Parameters:
        mapper - a ValueMapper that computes a new output value
        queryableStoreName - a user-provided name of the underlying KTable that can be used to subsequently query the operation results; valid characters are ASCII alphanumerics, '.', '_' and '-'. If null then the results cannot be queried (i.e., that would be equivalent to calling mapValues(ValueMapper).
        valueSerde - serializer for new value type
        Returns:
        a KTable that 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)
        Create a new KTable by transforming the value of each record in this KTable into a new value (with possible new type) in the new KTable. For each KTable update the provided ValueMapper is applied to the value of the updated record and computes a new value for it, resulting in an updated record for the result KTable. 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 KeyValueStore representing outputTable above it must be obtained via KafkaStreams#store(...): For non-local keys, a custom RPC mechanism must be implemented using KafkaStreams.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 mapValues for a changelog stream works differently than record stream filters, because records with null values (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 result KTable.

        Type Parameters:
        VR - the value type of the result KTable
        Parameters:
        mapper - a ValueMapper that computes a new output value
        valueSerde - serializer for new value type
        storeSupplier - user defined state store supplier. Cannot be null.
        Returns:
        a KTable that 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 by ReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to a KStream using toStream() and then use print(Printed.toSysOut()) on the result.
        Print the updated records of this KTable to System.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 calling toString() on the deserialized object.

        Implementors will need to override toString() for keys and values that are not of type String, Integer etc. to get meaningful information.

        Note that print() is not applied to the internal state store and only called for each new KTable updated record.

      • print

        @Deprecated
        void print​(java.lang.String label)
        Deprecated.
        Use the Interactive Queries APIs (e.g., KafkaStreams.store(String, QueryableStoreType) followed by ReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to a KStream using toStream() and then use print(Printed.toSysOut().withLabel(lable)) on the result.
        Print the updated records of this KTable to System.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 calling toString() on the deserialized object.

        Implementors will need to override toString() for keys and values that are not of type String, Integer etc. to get meaningful information.

        Note that print() is not applied to the internal state store and only called for each new KTable updated 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 by ReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to a KStream using toStream() and then use print(Printed.toSysOut().withKeyValueMapper(...) on the result.
        Print the updated records of this KTable to System.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 calling toString() on the deserialized object.

        Implementors will need to override toString() for keys and values that are not of type String, Integer etc. to get meaningful information.

        Note that print() is not applied to the internal state store and only called for each new KTable updated record.

        Parameters:
        keySerde - key serde used to deserialize key if type is byte[],
        valSerde - value serde used to deserialize value if type is byte[]
      • 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 by ReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to a KStream using toStream() and then use print(Printed.toSysOut().withLabel(label).withKeyValueMapper(...) on the result.
        Print the updated records of this KTable to System.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 calling toString() on the deserialized object.

        Implementors will need to override toString() for keys and values that are not of type String, Integer etc. to get meaningful information.

        Note that print() is not applied to the internal state store and only called for each new KTable updated record.

        Parameters:
        keySerde - key serde used to deserialize key if type is byte[],
        valSerde - value serde used to deserialize value if type is byte[],
        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 by ReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to a KStream using toStream() and then use print(Printed.toFile(filePath) on the result.
        Write the updated records of this KTable to 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 calling toString() on the deserialized object.

        Implementors will need to override toString() for keys and values that are not of type String, Integer etc. to get meaningful information.

        Note that writeAsText() is not applied to the internal state store and only called for each new KTable updated 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 by ReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to a KStream using toStream() and then use print(Printed.toFile(filePath).withLabel(label) on the result.
        Write the updated records of this KTable to 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 calling toString() on the deserialized object.

        Implementors will need to override toString() for keys and values that are not of type String, Integer etc. to get meaningful information.

        Note that writeAsText() is not applied to the internal state store and only called for each new KTable updated record.

        Parameters:
        filePath - name of file to write to
        label - 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 by ReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to a KStream using toStream() and then use print(Printed.toFile(filePath).withKeyValueMapper(...) on the result.
        Write the updated records of this KTable to 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 calling toString() on the deserialized object.

        Implementors will need to override toString() for keys and values that are not of type String, Integer etc. to get meaningful information.

        Note that writeAsText() is not applied to the internal state store and only called for each new KTable updated record.

        Parameters:
        filePath - name of file to write to
        keySerde - key serde used to deserialize key if type is byte[],
        valSerde - value serde used to deserialize value if type is byte[]
      • 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 by ReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to a KStream using toStream() and then use print(Printed.toFile(filePath).withLabel(label).withKeyValueMapper(...) on the result.
        Write the updated records of this KTable to 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 calling toString() on the deserialized object.

        Implementors will need to override toString() for keys and values that are not of type String, Integer etc. to get meaningful information.

        Note that writeAsText() is not applied to the internal state store and only called for each new KTable updated record.

        Parameters:
        filePath - name of file to write to
        label - the name used to label the key/value pairs printed to the console
        keySerde - key serde used to deserialize key if type is byte[],
        valSerde - value serde used to deserialize value if type is byte[]
      • foreach

        @Deprecated
        void foreach​(ForeachAction<? super K,​? super V> action)
        Deprecated.
        Use the Interactive Queries APIs (e.g., KafkaStreams.store(String, QueryableStoreType) followed by ReadOnlyKeyValueStore.all()) to iterate over the keys of a KTable. Alternatively convert to a KStream using toStream() and then use foreach(action) on the result.
        Perform an action on each updated record of this KTable. 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 new KTable updated record.

        Parameters:
        action - an action to perform on each record
      • toStream

        KStream<K,​V> toStream()
        Convert this changelog stream to a KStream.

        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. KStream vs KTable).

        Returns:
        a KStream that contains the same records as this KTable
      • toStream

        <KR> KStream<KR,​V> toStream​(KeyValueMapper<? super K,​? super V,​? extends KR> mapper)
        Convert this changelog stream to a KStream using the given KeyValueMapper to select the new key.

        For example, you can compute the new key as the length of the value string.

        
         KTable<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();
             }
         });
         
        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 result 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. KStream vs KTable).

        Type Parameters:
        KR - the new key type of the result stream
        Parameters:
        mapper - a KeyValueMapper that computes a new key for each record
        Returns:
        a KStream that contains the same records as this KTable
      • through

        @Deprecated
        KTable<K,​V> through​(java.lang.String topic,
                                  java.lang.String queryableStoreName)
        Deprecated.
        Materialize this changelog stream to a topic and creates a new KTable from the topic using default serializers and deserializers and producer's DefaultPartitioner. 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) and KStreamBuilder#table(someTopicName, queryableStoreName).

        The resulting KTable will 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 name
        queryableStoreName - the state store name used for the result KTable; valid characters are ASCII alphanumerics, '.', '_' and '-'. If null this is the equivalent of through(String)
        Returns:
        a KTable that contains the exact same (and potentially repartitioned) records as this KTable
      • through

        @Deprecated
        KTable<K,​V> through​(java.lang.String topic)
        Deprecated.
        use toStream() followed by to(topic) and and StreamsBuilder#table(topic) to read back as a KTable
        Materialize this changelog stream to a topic and creates a new KTable from the topic using default serializers and deserializers and producer's DefaultPartitioner. 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) and KStreamBuilder#table(someTopicName).

        The resulting KTable will be materialized in a local state store with an internal store name (cf. KStreamBuilder.table(String))

        Parameters:
        topic - the topic name
        Returns:
        a KTable that contains the exact same (and potentially repartitioned) records as this KTable
      • through

        @Deprecated
        KTable<K,​V> through​(StreamPartitioner<? super K,​? super V> partitioner,
                                  java.lang.String topic)
        Deprecated.
        Materialize this changelog stream to a topic and creates a new KTable from the topic using default serializers and deserializers and a customizable StreamPartitioner to 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) and KStreamBuilder#table(someTopicName).

        The resulting KTable will 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's DefaultPartitioner will be used
        topic - the topic name
        Returns:
        a KTable that contains the exact same (and potentially repartitioned) records as this KTable
      • 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)
        Materialize this changelog stream to a topic and creates a new KTable from the topic. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).

        If keySerde provides a WindowedSerializer for the key WindowedStreamPartitioner is used—otherwise producer's DefaultPartitioner is used.

        This is equivalent to calling #to(keySerde, valueSerde, someTopicName) and StreamsBuilder#table(someTopicName, queryableStoreName).

        The resulting KTable will 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 used
        valSerde - value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be used
        topic - the topic name
        queryableStoreName - the state store name used for the result KTable. If null this is the equivalent of through(Serde, Serde, String)
        Returns:
        a KTable that contains the exact same (and potentially repartitioned) records as this KTable
      • 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)
        Materialize this changelog stream to a topic and creates a new KTable from the topic. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).

        If keySerde provides a WindowedSerializer for the key WindowedStreamPartitioner is used—otherwise producer's DefaultPartitioner is used.

        This is equivalent to calling #to(keySerde, valueSerde, someTopicName) and KStreamBuilder#table(someTopicName, queryableStoreName).

        The resulting KTable will 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 used
        valSerde - value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be used
        topic - the topic name
        storeSupplier - user defined state store supplier. Cannot be null.
        Returns:
        a KTable that contains the exact same (and potentially repartitioned) records as this KTable
      • 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.
        Materialize this changelog stream to a topic and creates a new KTable from the topic. The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is started).

        If keySerde provides a WindowedSerializer for the key WindowedStreamPartitioner is used—otherwise producer's DefaultPartitioner is used.

        This is equivalent to calling #to(keySerde, valueSerde, someTopicName) and KStreamBuilder#table(someTopicName).

        The resulting KTable will 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 used
        valSerde - value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be used
        topic - the topic name
        Returns:
        a KTable that contains the exact same (and potentially repartitioned) records as this KTable
      • 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)
        Materialize this changelog stream to a topic and creates a new KTable from the topic using a customizable StreamPartitioner to 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) and KStreamBuilder#table(someTopicName, queryableStoreName).

        The resulting KTable will 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 used
        valSerde - value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be used
        partitioner - the function used to determine how records are distributed among partitions of the topic, if not specified and keySerde provides a WindowedSerializer for the key WindowedStreamPartitioner will be used—otherwise DefaultPartitioner will be used
        topic - the topic name
        queryableStoreName - the state store name used for the result KTable. If null this is the equivalent of {@link KTable#through(Serde, Serde, StreamPartitioner, String)()}
        Returns:
        a KTable that contains the exact same (and potentially repartitioned) records as this KTable
      • 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)
        Materialize this changelog stream to a topic and creates a new KTable from the topic using a customizable StreamPartitioner to 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) and KStreamBuilder#table(someTopicName, queryableStoreName).

        The resulting KTable will 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 used
        valSerde - value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be used
        partitioner - the function used to determine how records are distributed among partitions of the topic, if not specified and keySerde provides a WindowedSerializer for the key WindowedStreamPartitioner will be used—otherwise DefaultPartitioner will be used
        topic - the topic name
        storeSupplier - user defined state store supplier. Cannot be null.
        Returns:
        a KTable that contains the exact same (and potentially repartitioned) records as this KTable
      • 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)
        Materialize this changelog stream to a topic and creates a new KTable from the topic using a customizable StreamPartitioner to 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) and KStreamBuilder#table(someTopicName).

        The resulting KTable will 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 used
        valSerde - value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be used
        partitioner - the function used to determine how records are distributed among partitions of the topic, if not specified and keySerde provides a WindowedSerializer for the key WindowedStreamPartitioner will be used—otherwise DefaultPartitioner will be used
        topic - the topic name
        Returns:
        a KTable that contains the exact same (and potentially repartitioned) records as this KTable
      • to

        @Deprecated
        void to​(java.lang.String topic)
        Deprecated.
        use toStream() followed by to(topic)
        Materialize this changelog stream to a topic using default serializers and deserializers and producer's DefaultPartitioner. 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)
        Materialize this changelog stream to a topic using default serializers and deserializers and a customizable StreamPartitioner to 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's DefaultPartitioner will be used
        topic - 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)
        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 keySerde provides a WindowedSerializer for the key WindowedStreamPartitioner is used—otherwise producer's DefaultPartitioner is used.

        Parameters:
        keySerde - key serde used to send key-value pairs, if not specified the default key serde defined in the configuration will be used
        valSerde - value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be used
        topic - 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)
        Materialize this changelog stream to a topic using a customizable StreamPartitioner to 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 used
        valSerde - value serde used to send key-value pairs, if not specified the default value serde defined in the configuration will be used
        partitioner - the function used to determine how records are distributed among partitions of the topic, if not specified and keySerde provides a WindowedSerializer for the key WindowedStreamPartitioner will be used—otherwise DefaultPartitioner will be used
        topic - the topic name
      • groupBy

        <KR,​VR> KGroupedTable<KR,​VR> groupBy​(KeyValueMapper<? super K,​? super V,​KeyValue<KR,​VR>> selector)
        Re-groups the records of this KTable using the provided KeyValueMapper and default serializers and deserializers. Each KeyValue pair of this KTable is mapped to a new KeyValue pair by applying the provided KeyValueMapper. Re-grouping a KTable is required before an aggregation operator can be applied to the data (cf. KGroupedTable). The KeyValueMapper selects a new key and value (with should both have unmodified type). If the new record key is null the record will not be included in the resulting KGroupedTable

        Because 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 StreamsConfig via parameter APPLICATION_ID_CONFIG, "XXX" is an internally generated name, and "-repartition" is a fixed suffix. You can retrieve all generated internal topic names via KafkaStreams.toString().

        All data of this KTable will be redistributed through the repartitioning topic by writing all update records to and rereading all updated records from it, such that the resulting KGroupedTable is 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 result KGroupedTable
        VR - the value type of the result KGroupedTable
        Parameters:
        selector - a KeyValueMapper that computes a new grouping key and value to be aggregated
        Returns:
        a KGroupedTable that contains the re-grouped records of the original KTable
      • 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 this KTable using the provided KeyValueMapper and Serdes as specified by Serialized. Each KeyValue pair of this KTable is mapped to a new KeyValue pair by applying the provided KeyValueMapper. Re-grouping a KTable is required before an aggregation operator can be applied to the data (cf. KGroupedTable). The KeyValueMapper selects a new key and value (with should both have unmodified type). If the new record key is null the record will not be included in the resulting KGroupedTable

        Because 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 StreamsConfig via parameter APPLICATION_ID_CONFIG, "XXX" is an internally generated name, and "-repartition" is a fixed suffix. You can retrieve all generated internal topic names via KafkaStreams.toString().

        All data of this KTable will be redistributed through the repartitioning topic by writing all update records to and rereading all updated records from it, such that the resulting KGroupedTable is partitioned on the new key.

        Type Parameters:
        KR - the key type of the result KGroupedTable
        VR - the value type of the result KGroupedTable
        Parameters:
        selector - a KeyValueMapper that computes a new grouping key and value to be aggregated
        serialized - the Serialized instance used to specify Serdes
        Returns:
        a KGroupedTable that contains the re-grouped records of the original KTable
      • 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)
        Re-groups the records of this KTable using the provided KeyValueMapper. Each KeyValue pair of this KTable is mapped to a new KeyValue pair by applying the provided KeyValueMapper. Re-grouping a KTable is required before an aggregation operator can be applied to the data (cf. KGroupedTable). The KeyValueMapper selects a new key and value (both with potentially different type). If the new record key is null the record will not be included in the resulting KGroupedTable

        Because 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 StreamsConfig via parameter APPLICATION_ID_CONFIG, "XXX" is an internally generated name, and "-repartition" is a fixed suffix. You can retrieve all generated internal topic names via KafkaStreams.toString().

        All data of this KTable will be redistributed through the repartitioning topic by writing all update records to and rereading all updated records from it, such that the resulting KGroupedTable is partitioned on the new key.

        Type Parameters:
        KR - the key type of the result KGroupedTable
        VR - the value type of the result KGroupedTable
        Parameters:
        selector - a KeyValueMapper that computes a new grouping key and value to be aggregated
        keySerde - key serdes for materializing this stream, if not specified the default serdes defined in the configs will be used
        valueSerde - value serdes for materializing this stream, if not specified the default serdes defined in the configs will be used
        Returns:
        a KGroupedTable that contains the re-grouped records of the original KTable
      • join

        <VO,​VR> KTable<K,​VR> join​(KTable<K,​VO> other,
                                              ValueJoiner<? super V,​? super VO,​? extends VR> joiner)
        Join records of this KTable with another KTable's records using non-windowed inner equi join. The join is a primary key join with join attribute thisKTable.key == otherKTable.key. The result is an ever updating KTable that represents the current (i.e., processing time) result of the join.

        The join is computed by (1) updating the internal state of one KTable and (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the other KTable. This happens in a symmetric way, i.e., for each update of either this or the other input KTable the result gets updated.

        For each KTable record that finds a corresponding record in the other KTable the provided ValueJoiner will 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 records with null values (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 result KTable if required (i.e., if there is anything to be deleted).

        Input records with null key will be dropped and no join computation is performed.

        Example:

        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>
        Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.
        Type Parameters:
        VO - the value type of the other KTable
        VR - the value type of the result KTable
        Parameters:
        other - the other KTable to be joined with this KTable
        joiner - a ValueJoiner that computes the join result for a pair of matching records
        Returns:
        a KTable that contains join-records for each key and values computed by the given ValueJoiner, 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 this KTable with another KTable's records using non-windowed inner equi join. The join is a primary key join with join attribute thisKTable.key == otherKTable.key. The result is an ever updating KTable that represents the current (i.e., processing time) result of the join.

        The join is computed by (1) updating the internal state of one KTable and (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the other KTable. This happens in a symmetric way, i.e., for each update of either this or the other input KTable the result gets updated.

        For each KTable record that finds a corresponding record in the other KTable the provided ValueJoiner will 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 records with null values (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 result KTable if required (i.e., if there is anything to be deleted).

        Input records with null key will be dropped and no join computation is performed.

        Example:

        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>
        Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.
        Type Parameters:
        VO - the value type of the other KTable
        VR - the value type of the result KTable
        Parameters:
        other - the other KTable to be joined with this KTable
        joiner - a ValueJoiner that computes the join result for a pair of matching records
        materialized - an instance of Materialized used to describe how the state store should be materialized. Cannot be null
        Returns:
        a KTable that contains join-records for each key and values computed by the given ValueJoiner, 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 this KTable with another KTable's records using non-windowed inner equi join. The join is a primary key join with join attribute thisKTable.key == otherKTable.key. The result is an ever updating KTable that represents the current (i.e., processing time) result of the join.

        The join is computed by (1) updating the internal state of one KTable and (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the other KTable. This happens in a symmetric way, i.e., for each update of either this or the other input KTable the result gets updated.

        For each KTable record that finds a corresponding record in the other KTable the provided ValueJoiner will 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 records with null values (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 result KTable if required (i.e., if there is anything to be deleted).

        Input records with null key will be dropped and no join computation is performed.

        Example:

        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>
        Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.
        Type Parameters:
        VO - the value type of the other KTable
        VR - the value type of the result KTable
        Parameters:
        other - the other KTable to be joined with this KTable
        joiner - a ValueJoiner that computes the join result for a pair of matching records
        joinSerde - serializer for join result value type
        queryableStoreName - a user-provided name of the underlying KTable that can be used to subsequently query the operation results; valid characters are ASCII alphanumerics, '.', '_' and '-'. If null then the results cannot be queried (i.e., that would be equivalent to calling join(KTable, ValueJoiner).
        Returns:
        a KTable that contains join-records for each key and values computed by the given ValueJoiner, 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)
        Join records of this KTable with another KTable's records using non-windowed inner equi join. The join is a primary key join with join attribute thisKTable.key == otherKTable.key. The result is an ever updating KTable that represents the current (i.e., processing time) result of the join.

        The join is computed by (1) updating the internal state of one KTable and (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the other KTable. This happens in a symmetric way, i.e., for each update of either this or the other input KTable the result gets updated.

        For each KTable record that finds a corresponding record in the other KTable the provided ValueJoiner will 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 records with null values (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 result KTable if required (i.e., if there is anything to be deleted).

        Input records with null key will be dropped and no join computation is performed.

        Example:

        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>
        Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.
        Type Parameters:
        VO - the value type of the other KTable
        VR - the value type of the result KTable
        Parameters:
        other - the other KTable to be joined with this KTable
        joiner - a ValueJoiner that computes the join result for a pair of matching records
        storeSupplier - user defined state store supplier. Cannot be null.
        Returns:
        a KTable that contains join-records for each key and values computed by the given ValueJoiner, 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 this KTable (left input) with another KTable's (right input) records using non-windowed left equi join. The join is a primary key join with join attribute thisKTable.key == otherKTable.key. In contrast to inner-join, all records from left KTable will produce an output record (cf. below). The result is an ever updating KTable that represents the current (i.e., processing time) result of the join.

        The join is computed by (1) updating the internal state of one KTable and (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the other KTable. This happens in a symmetric way, i.e., for each update of either this or the other input KTable the result gets updated.

        For each KTable record that finds a corresponding record in the other KTable's state the provided ValueJoiner will be called to compute a value (with arbitrary type) for the result record. Additionally, for each record of left KTable that does not find a corresponding record in the right KTable's state the provided ValueJoiner will be called with rightValue = null 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 records with null values (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 result KTable if required (i.e., if there is anything to be deleted).

        Input records with null key will be dropped and no join computation is performed.

        Example:

        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>
        Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.
        Type Parameters:
        VO - the value type of the other KTable
        VR - the value type of the result KTable
        Parameters:
        other - the other KTable to be joined with this KTable
        joiner - a ValueJoiner that computes the join result for a pair of matching records
        Returns:
        a KTable that contains join-records for each key and values computed by the given ValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of left KTable
        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 this KTable (left input) with another KTable's (right input) records using non-windowed left equi join. The join is a primary key join with join attribute thisKTable.key == otherKTable.key. In contrast to inner-join, all records from left KTable will produce an output record (cf. below). The result is an ever updating KTable that represents the current (i.e., processing time) result of the join.

        The join is computed by (1) updating the internal state of one KTable and (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the other KTable. This happens in a symmetric way, i.e., for each update of either this or the other input KTable the result gets updated.

        For each KTable record that finds a corresponding record in the other KTable's state the provided ValueJoiner will be called to compute a value (with arbitrary type) for the result record. Additionally, for each record of left KTable that does not find a corresponding record in the right KTable's state the provided ValueJoiner will be called with rightValue = null 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 records with null values (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 result KTable if required (i.e., if there is anything to be deleted).

        Input records with null key will be dropped and no join computation is performed.

        Example:

        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>
        Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.
        Type Parameters:
        VO - the value type of the other KTable
        VR - the value type of the result KTable
        Parameters:
        other - the other KTable to be joined with this KTable
        joiner - a ValueJoiner that computes the join result for a pair of matching records
        materialized - an instance of Materialized used to describe how the state store should be materialized. Cannot be null
        Returns:
        a KTable that contains join-records for each key and values computed by the given ValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of left KTable
        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)
        Join records of this KTable (left input) with another KTable's (right input) records using non-windowed left equi join. The join is a primary key join with join attribute thisKTable.key == otherKTable.key. In contrast to inner-join, all records from left KTable will produce an output record (cf. below). The result is an ever updating KTable that represents the current (i.e., processing time) result of the join.

        The join is computed by (1) updating the internal state of one KTable and (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the other KTable. This happens in a symmetric way, i.e., for each update of either this or the other input KTable the result gets updated.

        For each KTable record that finds a corresponding record in the other KTable's state the provided ValueJoiner will be called to compute a value (with arbitrary type) for the result record. Additionally, for each record of left KTable that does not find a corresponding record in the right KTable's state the provided ValueJoiner will be called with rightValue = null 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 records with null values (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 result KTable if required (i.e., if there is anything to be deleted).

        Input records with null key will be dropped and no join computation is performed.

        Example:

        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>
        Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.
        Type Parameters:
        VO - the value type of the other KTable
        VR - the value type of the result KTable
        Parameters:
        other - the other KTable to be joined with this KTable
        joiner - a ValueJoiner that computes the join result for a pair of matching records
        joinSerde - serializer for join result value type
        queryableStoreName - a user-provided name of the underlying KTable that can be used to subsequently query the operation results; valid characters are ASCII alphanumerics, '.', '_' and '-'. If null then the results cannot be queried (i.e., that would be equivalent to calling leftJoin(KTable, ValueJoiner).
        Returns:
        a KTable that contains join-records for each key and values computed by the given ValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of left KTable
        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 this KTable (left input) with another KTable's (right input) records using non-windowed left equi join. The join is a primary key join with join attribute thisKTable.key == otherKTable.key. In contrast to inner-join, all records from left KTable will produce an output record (cf. below). The result is an ever updating KTable that represents the current (i.e., processing time) result of the join.

        The join is computed by (1) updating the internal state of one KTable and (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the other KTable. This happens in a symmetric way, i.e., for each update of either this or the other input KTable the result gets updated.

        For each KTable record that finds a corresponding record in the other KTable's state the provided ValueJoiner will be called to compute a value (with arbitrary type) for the result record. Additionally, for each record of left KTable that does not find a corresponding record in the right KTable's state the provided ValueJoiner will be called with rightValue = null 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 records with null values (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 result KTable if required (i.e., if there is anything to be deleted).

        Input records with null key will be dropped and no join computation is performed.

        Example:

        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>
        Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.
        Type Parameters:
        VO - the value type of the other KTable
        VR - the value type of the result KTable
        Parameters:
        other - the other KTable to be joined with this KTable
        joiner - a ValueJoiner that computes the join result for a pair of matching records
        storeSupplier - user defined state store supplier. Cannot be null.
        Returns:
        a KTable that contains join-records for each key and values computed by the given ValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of left KTable
        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 this KTable (left input) with another KTable's (right input) records using non-windowed outer equi join. The join is a primary key join with join attribute thisKTable.key == otherKTable.key. In contrast to inner-join or left-join, all records from both input KTables will produce an output record (cf. below). The result is an ever updating KTable that represents the current (i.e., processing time) result of the join.

        The join is computed by (1) updating the internal state of one KTable and (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the other KTable. This happens in a symmetric way, i.e., for each update of either this or the other input KTable the result gets updated.

        For each KTable record that finds a corresponding record in the other KTable's state the provided ValueJoiner will 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 other KTable's state the provided ValueJoiner will be called with null value 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 records with null values (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 result KTable if required (i.e., if there is anything to be deleted).

        Input records with null key will be dropped and no join computation is performed.

        Example:

        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>
        Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.
        Type Parameters:
        VO - the value type of the other KTable
        VR - the value type of the result KTable
        Parameters:
        other - the other KTable to be joined with this KTable
        joiner - a ValueJoiner that computes the join result for a pair of matching records
        Returns:
        a KTable that contains join-records for each key and values computed by the given ValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of both KTables
        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 this KTable (left input) with another KTable's (right input) records using non-windowed outer equi join. The join is a primary key join with join attribute thisKTable.key == otherKTable.key. In contrast to inner-join or left-join, all records from both input KTables will produce an output record (cf. below). The result is an ever updating KTable that represents the current (i.e., processing time) result of the join.

        The join is computed by (1) updating the internal state of one KTable and (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the other KTable. This happens in a symmetric way, i.e., for each update of either this or the other input KTable the result gets updated.

        For each KTable record that finds a corresponding record in the other KTable's state the provided ValueJoiner will 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 other KTable's state the provided ValueJoiner will be called with null value 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 records with null values (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 result KTable if required (i.e., if there is anything to be deleted).

        Input records with null key will be dropped and no join computation is performed.

        Example:

        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>
        Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.
        Type Parameters:
        VO - the value type of the other KTable
        VR - the value type of the result KTable
        Parameters:
        other - the other KTable to be joined with this KTable
        joiner - a ValueJoiner that computes the join result for a pair of matching records
        materialized - an instance of Materialized used to describe how the state store should be materialized. Cannot be null
        Returns:
        a KTable that contains join-records for each key and values computed by the given ValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of both KTables
        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)
        Join records of this KTable (left input) with another KTable's (right input) records using non-windowed outer equi join. The join is a primary key join with join attribute thisKTable.key == otherKTable.key. In contrast to inner-join or left-join, all records from both input KTables will produce an output record (cf. below). The result is an ever updating KTable that represents the current (i.e., processing time) result of the join.

        The join is computed by (1) updating the internal state of one KTable and (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the other KTable. This happens in a symmetric way, i.e., for each update of either this or the other input KTable the result gets updated.

        For each KTable record that finds a corresponding record in the other KTable's state the provided ValueJoiner will 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 other KTable's state the provided ValueJoiner will be called with null value 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 records with null values (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 result KTable if required (i.e., if there is anything to be deleted).

        Input records with null key will be dropped and no join computation is performed.

        Example:

        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>
        Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.
        Type Parameters:
        VO - the value type of the other KTable
        VR - the value type of the result KTable
        Parameters:
        other - the other KTable to be joined with this KTable
        joiner - a ValueJoiner that computes the join result for a pair of matching records
        joinSerde - serializer for join result value type
        queryableStoreName - a user-provided name of the underlying KTable that can be used to subsequently query the operation results; valid characters are ASCII alphanumerics, '.', '_' and '-'. If null then the results cannot be queried (i.e., that would be equivalent to calling outerJoin(KTable, ValueJoiner).
        Returns:
        a KTable that contains join-records for each key and values computed by the given ValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of both KTables
        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 this KTable (left input) with another KTable's (right input) records using non-windowed outer equi join. The join is a primary key join with join attribute thisKTable.key == otherKTable.key. In contrast to inner-join or left-join, all records from both input KTables will produce an output record (cf. below). The result is an ever updating KTable that represents the current (i.e., processing time) result of the join.

        The join is computed by (1) updating the internal state of one KTable and (2) performing a lookup for a matching record in the current (i.e., processing time) internal state of the other KTable. This happens in a symmetric way, i.e., for each update of either this or the other input KTable the result gets updated.

        For each KTable record that finds a corresponding record in the other KTable's state the provided ValueJoiner will 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 other KTable's state the provided ValueJoiner will be called with null value 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 records with null values (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 result KTable if required (i.e., if there is anything to be deleted).

        Input records with null key will be dropped and no join computation is performed.

        Example:

        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>
        Both input streams (or to be more precise, their underlying source topics) need to have the same number of partitions.
        Type Parameters:
        VO - the value type of the other KTable
        VR - the value type of the result KTable
        Parameters:
        other - the other KTable to be joined with this KTable
        joiner - a ValueJoiner that computes the join result for a pair of matching records
        storeSupplier - user defined state store supplier. Cannot be null.
        Returns:
        a KTable that contains join-records for each key and values computed by the given ValueJoiner, one for each matched record-pair with the same key plus one for each non-matching record of both KTables
        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 this KTable.
        Returns:
        the underlying state store name, or null if this KTable cannot be queried.