KAFKA-6905: Document that Transformers may be re-used by Streams (#5026)

This is a follow-up to #5022 which added documentation to the Processor
interface. This commit adds similar documentation to Transformer and
ValueTransformer.

Also, s/processor/transformer/ in the close() docs.

Reviewers: Matthias J. Sax <matthias@confluent.io>, Guozhang Wang <wangguoz@gmail.com>
This commit is contained in:
David Glasser 2018-05-19 18:44:20 -07:00 committed by Guozhang Wang
parent 4e1c8ffd0d
commit f65f3a878f
2 changed files with 8 additions and 2 deletions

View File

@ -51,6 +51,8 @@ public interface Transformer<K, V, R> {
/**
* Initialize this transformer.
* This is called once per instance when the topology gets initialized.
* When the framework is done with the transformer, {@link #close()} will be called on it; the
* framework may later re-use the transformer by calling {@link #init()} again.
* <p>
* The provided {@link ProcessorContext context} can be used to access topology and record meta data, to
* {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} a method to be
@ -81,7 +83,8 @@ public interface Transformer<K, V, R> {
R transform(final K key, final V value);
/**
* Close this processor and clean up any resources.
* Close this transformer and clean up any resources. The framework may
* later re-use this transformer by calling {@link #init()} on it again.
* <p>
* To generate new {@link KeyValue} pairs {@link ProcessorContext#forward(Object, Object)} and
* {@link ProcessorContext#forward(Object, Object, To)} can be used.

View File

@ -50,6 +50,8 @@ public interface ValueTransformer<V, VR> {
/**
* Initialize this transformer.
* This is called once per instance when the topology gets initialized.
* When the framework is done with the transformer, {@link #close()} will be called on it; the
* framework may later re-use the transformer by calling {@link #init()} again.
* <p>
* The provided {@link ProcessorContext context} can be used to access topology and record meta data, to
* {@link ProcessorContext#schedule(long, PunctuationType, Punctuator) schedule} a method to be
@ -84,7 +86,8 @@ public interface ValueTransformer<V, VR> {
VR transform(final V value);
/**
* Close this processor and clean up any resources.
* Close this transformer and clean up any resources. The framework may
* later re-use this transformer by calling {@link #init()} on it again.
* <p>
* It is not possible to return any new output records within {@code close()}.
* Using {@link ProcessorContext#forward(Object, Object)} or {@link ProcessorContext#forward(Object, Object, To)}