Browse Source

MINOR: clarify statefulness in transform/process variations (#7668)

Reviewer: Almog Gavra <almog@confluent.io>, Bill Bejeck <bill@confluent.io>, Matthias J. Sax <matthias@confluent.io>
pull/7676/head
A. Sophie Blee-Goldman 5 years ago committed by Matthias J. Sax
parent
commit
1cd5d0da1d
  1. 69
      streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java

69
streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java

@ -879,7 +879,10 @@ public interface KStream<K, V> { @@ -879,7 +879,10 @@ public interface KStream<K, V> {
* A {@link Transformer} (provided by the given {@link TransformerSupplier}) is applied to each input record and
* returns zero or one output record.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K':V'>}.
* This is a stateful record-by-record operation (cf. {@link #map(KeyValueMapper) map()}).
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #map(KeyValueMapper) map()}).
* If you choose not to attach one, this operation is similar to the stateless {@link #map(KeyValueMapper) map()}
* but allows access to the {@code ProcessorContext} and record metadata.
* This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()},
* the processing progress can be observed and additional periodic actions can be performed.
* <p>
@ -970,7 +973,10 @@ public interface KStream<K, V> { @@ -970,7 +973,10 @@ public interface KStream<K, V> {
* A {@link Transformer} (provided by the given {@link TransformerSupplier}) is applied to each input record and
* returns zero or one output record.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K':V'>}.
* This is a stateful record-by-record operation (cf. {@link #map(KeyValueMapper) map()}).
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #map(KeyValueMapper) map()}).
* If you choose not to attach one, this operation is similar to the stateless {@link #map(KeyValueMapper) map()}
* but allows access to the {@code ProcessorContext} and record metadata.
* This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()},
* the processing progress can be observed and additional periodic actions can be performed.
* <p>
@ -1063,8 +1069,9 @@ public interface KStream<K, V> { @@ -1063,8 +1069,9 @@ public interface KStream<K, V> {
* A {@link Transformer} (provided by the given {@link TransformerSupplier}) is applied to each input record and
* returns zero or more output records.
* Thus, an input record {@code <K,V>} can be transformed into output records {@code <K':V'>, <K'':V''>, ...}.
* This is a stateful record-by-record operation (cf. {@link #flatMap(KeyValueMapper) flatMap()} for stateless
* record transformation).
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #flatMap(KeyValueMapper) flatMap()).
* If you choose not to attach one, this operation is similar to the stateless {@link #flatMap(KeyValueMapper) flatMap()
* but allows access to the {@code ProcessorContext} and record metadata.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}
* the processing progress can be observed and additional periodic actions can be performed.
* <p>
@ -1155,8 +1162,9 @@ public interface KStream<K, V> { @@ -1155,8 +1162,9 @@ public interface KStream<K, V> {
* A {@link Transformer} (provided by the given {@link TransformerSupplier}) is applied to each input record and
* returns zero or more output records.
* Thus, an input record {@code <K,V>} can be transformed into output records {@code <K':V'>, <K'':V''>, ...}.
* This is a stateful record-by-record operation (cf. {@link #flatMap(KeyValueMapper) flatMap()} for stateless
* record transformation).
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #flatMap(KeyValueMapper) flatMap()}).
* If you choose not to attach one, this operation is similar to the stateless {@link #flatMap(KeyValueMapper) flatMap()}
* but allows access to the {@code ProcessorContext} and record metadata.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}
* the processing progress can be observed and additional periodic actions can be performed.
* <p>
@ -1248,7 +1256,9 @@ public interface KStream<K, V> { @@ -1248,7 +1256,9 @@ public interface KStream<K, V> {
* A {@link ValueTransformer} (provided by the given {@link ValueTransformerSupplier}) is applied to each input
* record value and computes a new value for it.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
* This is a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper)}).
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper) mapValues()}).
* If you choose not to attach one, this operation is similar to the stateless {@link #mapValues(ValueMapper) mapValues()}
* but allows access to the {@code ProcessorContext} and record metadata.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress
* can be observed and additional periodic actions can be performed.
* <p>
@ -1323,7 +1333,10 @@ public interface KStream<K, V> { @@ -1323,7 +1333,10 @@ public interface KStream<K, V> {
* A {@link ValueTransformer} (provided by the given {@link ValueTransformerSupplier}) is applied to each input
* record value and computes a new value for it.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
* This is a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper)}).
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper) mapValues()}).
* If you choose not to attach one, this operation is similar to the stateless {@link #mapValues(ValueMapper) mapValues()}
* but allows access to the {@code ProcessorContext} and record metadata.
* This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress
* can be observed and additional periodic actions can be performed.
* <p>
@ -1401,7 +1414,10 @@ public interface KStream<K, V> { @@ -1401,7 +1414,10 @@ public interface KStream<K, V> {
* A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applied to
* each input record value and computes a new value for it.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
* This is a stateful record-by-record operation (cf. {@link #mapValues(ValueMapperWithKey)}).
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #mapValues(ValueMapperWithKey) mapValues()}).
* If you choose not to attach one, this operation is similar to the stateless {@link #mapValues(ValueMapperWithKey) mapValues()}
* but allows access to the {@code ProcessorContext} and record metadata.
* This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress
* can be observed and additional periodic actions can be performed.
* <p>
@ -1480,7 +1496,10 @@ public interface KStream<K, V> { @@ -1480,7 +1496,10 @@ public interface KStream<K, V> {
* A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applied to
* each input record value and computes a new value for it.
* Thus, an input record {@code <K,V>} can be transformed into an output record {@code <K:V'>}.
* This is a stateful record-by-record operation (cf. {@link #mapValues(ValueMapperWithKey)}).
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #mapValues(ValueMapperWithKey) mapValues()}).
* If you choose not to attach one, this operation is similar to the stateless {@link #mapValues(ValueMapperWithKey) mapValues()}
* but allows access to the {@code ProcessorContext} and record metadata.
* This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress
* can be observed and additional periodic actions can be performed.
* <p>
@ -1561,7 +1580,10 @@ public interface KStream<K, V> { @@ -1561,7 +1580,10 @@ public interface KStream<K, V> {
* A {@link ValueTransformer} (provided by the given {@link ValueTransformerSupplier}) is applied to each input
* record value and computes zero or more new values.
* Thus, an input record {@code <K,V>} can be transformed into output records {@code <K:V'>, <K:V''>, ...}.
* This is a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper) mapValues()}).
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper) mapValues()}).
* If you choose not to attach one, this operation is similar to the stateless {@link #mapValues(ValueMapper) mapValues()}
* but allows access to the {@code ProcessorContext} and record metadata.
* This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}
* the processing progress can be observed and additional periodic actions can be performed.
* <p>
@ -1647,7 +1669,10 @@ public interface KStream<K, V> { @@ -1647,7 +1669,10 @@ public interface KStream<K, V> {
* A {@link ValueTransformer} (provided by the given {@link ValueTransformerSupplier}) is applied to each input
* record value and computes zero or more new values.
* Thus, an input record {@code <K,V>} can be transformed into output records {@code <K:V'>, <K:V''>, ...}.
* This is a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper) mapValues()}).
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper) mapValues()}).
* If you choose not to attach one, this operation is similar to the stateless {@link #mapValues(ValueMapper) mapValues()}
* but allows access to the {@code ProcessorContext} and record metadata.
* This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}
* the processing progress can be observed and additional periodic actions can be performed.
* <p>
@ -1735,7 +1760,10 @@ public interface KStream<K, V> { @@ -1735,7 +1760,10 @@ public interface KStream<K, V> {
* A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applied to
* each input record value and computes zero or more new values.
* Thus, an input record {@code <K,V>} can be transformed into output records {@code <K:V'>, <K:V''>, ...}.
* This is a stateful record-by-record operation (cf. {@link #flatMapValues(ValueMapperWithKey) flatMapValues()}).
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #flatMapValues(ValueMapperWithKey) flatMapValues()}).
* If you choose not to attach one, this operation is similar to the stateless {@link #flatMapValues(ValueMapperWithKey) flatMapValues()}
* but allows access to the {@code ProcessorContext} and record metadata.
* This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress can
* be observed and additional periodic actions can be performed.
* <p>
@ -1822,7 +1850,10 @@ public interface KStream<K, V> { @@ -1822,7 +1850,10 @@ public interface KStream<K, V> {
* A {@link ValueTransformerWithKey} (provided by the given {@link ValueTransformerWithKeySupplier}) is applied to
* each input record value and computes zero or more new values.
* Thus, an input record {@code <K,V>} can be transformed into output records {@code <K:V'>, <K:V''>, ...}.
* This is a stateful record-by-record operation (cf. {@link #flatMapValues(ValueMapperWithKey) flatMapValues()}).
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #flatMapValues(ValueMapperWithKey) flatMapValues()}).
* If you choose not to attach one, this operation is similar to the stateless {@link #flatMapValues(ValueMapperWithKey) flatMapValues()}
* but allows access to the {@code ProcessorContext} and record metadata.
* This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress can
* be observed and additional periodic actions can be performed.
* <p>
@ -1908,7 +1939,10 @@ public interface KStream<K, V> { @@ -1908,7 +1939,10 @@ public interface KStream<K, V> {
/**
* Process all records in this stream, one record at a time, by applying a {@link Processor} (provided by the given
* {@link ProcessorSupplier}).
* This is a stateful record-by-record operation (cf. {@link #foreach(ForeachAction)}).
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #foreach(ForeachAction)}).
* If you choose not to attach one, this operation is similar to the stateless {@link #foreach(ForeachAction)}
* but allows access to the {@code ProcessorContext} and record metadata.
* This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress
* can be observed and additional periodic actions can be performed.
* Note that this is a terminal operation that returns void.
@ -1967,7 +2001,10 @@ public interface KStream<K, V> { @@ -1967,7 +2001,10 @@ public interface KStream<K, V> {
/**
* Process all records in this stream, one record at a time, by applying a {@link Processor} (provided by the given
* {@link ProcessorSupplier}).
* This is a stateful record-by-record operation (cf. {@link #foreach(ForeachAction)}).
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #foreach(ForeachAction)}).
* If you choose not to attach one, this operation is similar to the stateless {@link #foreach(ForeachAction)}
* but allows access to the {@code ProcessorContext} and record metadata.
* This is essentially mixing the Processor API into the DSL, and provides all the functionality of the PAPI.
* Furthermore, via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long)} the processing progress
* can be observed and additional periodic actions can be performed.
* Note that this is a terminal operation that returns void.

Loading…
Cancel
Save