-
Notifications
You must be signed in to change notification settings - Fork 14.6k
MINOR: update JavaDoc for DSL PAPI-API #2413
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,48 +17,96 @@ | |
|
||
package org.apache.kafka.streams.kstream; | ||
|
||
import org.apache.kafka.common.annotation.InterfaceStability; | ||
import org.apache.kafka.streams.KeyValue; | ||
import org.apache.kafka.streams.errors.StreamsException; | ||
import org.apache.kafka.streams.processor.ProcessorContext; | ||
import org.apache.kafka.streams.processor.StateStore; | ||
import org.apache.kafka.streams.processor.TimestampExtractor; | ||
|
||
/** | ||
* A stateful {@link Transformer} interface to transform a key-value pair into a new value. | ||
* The {@link Transformer} interface for stateful mapping of an input record to zero, one, or multiple new output | ||
* records (both key and value type can be altered arbitrarily). | ||
* This is a stateful record-by-record operation, i.e, {@link #transform(Object, Object)} is invoked individually for | ||
* each record of a stream and can access and modify a state that is available beyond a single call of | ||
* {@link #transform(Object, Object)} (cf. {@link KeyValueMapper} for stateless record transformation). | ||
* Additionally, the interface can be called in regular intervals based on the processing progress | ||
* (cf. {@link #punctuate(long)}. | ||
* <p> | ||
* Use {@link TransformerSupplier} to provide new instances of {@link Transformer} to Kafka Stream's runtime. | ||
* <p> | ||
* If only a record's value should be modified {@link ValueTransformer} can be used. | ||
* | ||
* @param <K> key type | ||
* @param <V> value type | ||
* @param <R> return type | ||
* @param <K> key type | ||
* @param <V> value type | ||
* @param <R> {@link KeyValue} return type (both key and value type can be set | ||
* arbitrarily) | ||
* @see TransformerSupplier | ||
* @see KStream#transform(TransformerSupplier, String...) | ||
* @see ValueTransformer | ||
* @see KStream#map(KeyValueMapper) | ||
* @see KStream#flatMap(KeyValueMapper) | ||
*/ | ||
@InterfaceStability.Unstable | ||
public interface Transformer<K, V, R> { | ||
|
||
/** | ||
* Initialize this transformer with the given context. The framework ensures this is called once per processor when the topology | ||
* that contains it is initialized. | ||
* Initialize this transformer. | ||
* This is called once per instance when the topology gets initialized. | ||
* <p> | ||
* If this transformer is to be {@link #punctuate(long) called periodically} by the framework, then this method should | ||
* {@link ProcessorContext#schedule(long) schedule itself} with the provided context. | ||
* The provided {@link ProcessorContext context} can be used to access topology and record meta data, to | ||
* {@link ProcessorContext#schedule(long) schedule itself} for periodical calls (cf. {@link #punctuate(long)}), and | ||
* to access attached {@link StateStore}s. | ||
* <p> | ||
* Note, that {@link ProcessorContext} is updated in the background with the current record's meta data. | ||
* Thus, it only contains valid record meta data when accessed within {@link #transform(Object, Object)}. | ||
* | ||
* @param context the context; may not be null | ||
* @param context the context | ||
*/ | ||
void init(final ProcessorContext context); | ||
|
||
/** | ||
* Transform the record with the given key and value. | ||
* Additionally, any {@link StateStore state} that is {@link KStream#transform(TransformerSupplier, String...) | ||
* attached} to this operator can be accessed and modified | ||
* arbitrarily (cf. {@link ProcessorContext#getStateStore(String)}). | ||
* <p> | ||
* If more than one output record should be forwarded downstream {@link ProcessorContext#forward(Object, Object)}, | ||
* {@link ProcessorContext#forward(Object, Object, int)}, and | ||
* {@link ProcessorContext#forward(Object, Object, String)} can be used. | ||
* If not record should be forwarded downstream, {@code transform} can return {@code null}. | ||
* | ||
* @param key the key for the record | ||
* @param value the value for the record | ||
* @return new value; if null no key-value pair will be forwarded to down stream | ||
* @return new {@link KeyValue} pair—if {@code null} no key-value pair will | ||
* be forwarded to down stream | ||
*/ | ||
R transform(final K key, final V value); | ||
|
||
/** | ||
* Perform any periodic operations and possibly generate a key, if this processor {@link ProcessorContext#schedule(long) schedules itself} with the context | ||
* during {@link #init(ProcessorContext) initialization}. | ||
* Perform any periodic operations and possibly generate new {@link KeyValue} pairs if this processor | ||
* {@link ProcessorContext#schedule(long) schedules itself} with the context during | ||
* {@link #init(ProcessorContext) initialization}. | ||
* <p> | ||
* To generate new {@link KeyValue} pairs {@link ProcessorContext#forward(Object, Object)}, | ||
* {@link ProcessorContext#forward(Object, Object, int)}, and | ||
* {@link ProcessorContext#forward(Object, Object, String)} can be used. | ||
* <p> | ||
* Note that {@code punctuate} is called based on <it>stream time</it> (i.e., time progresses with regard to | ||
* timestamps return by the used {@link TimestampExtractor}) | ||
* and not based on wall-clock time. | ||
* | ||
* @param timestamp the stream time when this method is being called | ||
* @return new value; if null it will not be forwarded to down stream | ||
* @param timestamp the stream time when {@code punctuate} is being called | ||
* @return must return {@code null}—otherwise, a {@link StreamsException exception} will be thrown | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't think this change is correct. KStreamTransformProcessor does: There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks for reporting this @mihbor (I guess this was a mix up with There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks for the PR! |
||
*/ | ||
R punctuate(final long timestamp); | ||
|
||
/** | ||
* Close this processor and clean up any resources. | ||
* <p> | ||
* To generate new {@link KeyValue} pairs {@link ProcessorContext#forward(Object, Object)}, | ||
* {@link ProcessorContext#forward(Object, Object, int)}, and | ||
* {@link ProcessorContext#forward(Object, Object, String)} can be used. | ||
*/ | ||
void close(); | ||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Hmm good point. Maybe we should consider ignore nulls in maps if users found it not very friendly. Anyways, just a reminder for future changes.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I would not change anything with regard to
null
: if users want to change "cardinality" they should use "flatMap". If we want to do some reasoning about performance, it's good to know that map does not change cardinality: one in, one out.