Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Table of Contents

Status

Current stateUnder DiscussionAccepted

Discussion thread: here

JIRA: KAFKA-6849

...

The following methods would be added to the the Java KTable interface: 

Code Block
languagejava
titleNew KTable methods
linenumberstrue
<VR> KTable<K, VR> transformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, ? extends VR> valueTransformerSupplier,
                                   final String... stateStoreNames);
 
<VR> KTable<K, VR> transformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, ? extends VR> valueTransformerSupplier,
                                   final Materialized<K, VR, KeyValueStore<Bytes, byte[]>> materialized,
                                   final String... stateStoreNames);

The following methods would be added to the Scala KTable class.

Code Block
languagescala
titleAdditional Scala KTable methods
def transformValues[VR](valueTransformerSupplier: ValueTransformerWithKeySupplier[K, V, VR],
                        stateStoreNames: String*): KTable[K, VR]

def transformValues[VR](valueTransformerSupplier: ValueTransformerWithKeySupplier[K, V, VR],
                        materialized: Materialized[K, VR, KeyValueStore[Bytes, Array[Byte]]],
                        stateStoreNames: String*): KTable[K, VR]

 

Proposed Changes

The new methods on `KTableImpl` will add a new KTableTransformValues processor node and attach any state stores.  The new KTableTransformValues will be implemented in a similar manner to other processors, instantiating the user supplied transformer once per task.

...