Versions Compared

Key

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

...

Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

Motivation

 

Kafka Streams has a defined "contract" about timestamp propagation at the Processor API level: all processors within a sub-topology, see the timestamp from the input topic record that is currently processed and this timestamp will be used for all result records when writing them to a topic, too. For the DSL and also for custom operators, it would be desirable to allow timestamp manipulation at Processor API level for individual records that are forwarded. This allows to support a larger scope of possible semantics for stream processing.

...

Code Block
languagejava
package org.apache.kafka.stream.processor;
 
public class To {
  private To(String childName, int childIndex, long timestamp);


  public static To child(String childName);
  public static To child(int childIndex);
  public static To all();
 
  public To withTimestamp(long timestamp);
}

...

We add one new overload of ProcessorContext#forward() that take one additional parameter of type To. to specify optional argument like childName , childIndex, or output record timestamp. If users call the new overloads and set a timestamp on the To object, the output record gets the specified timestamp assigned. For the existing methods or if no timestamp is specified on To, the default contract using the input record timestamp for the output record will be used. Note, that the new To class does not include forwarding by index anymore—we want to remove forwarding by index and only support forwarding to all or by name.

Compatibility, Deprecation, and Migration Plan

...