mirror of https://github.com/apache/kafka.git
KAFKA-13654: Extend KStream process with new Processor API (#11993)
Updates the KStream process API to cover the use cases of both process and transform, and deprecate the KStream transform API. Implements KIP-820 Reviewer: John Roesler <vvcephei@apache.org>
This commit is contained in:
parent
9ec232fde8
commit
fa0324485b
|
@ -170,7 +170,7 @@
|
|||
|
||||
<!-- Streams -->
|
||||
<suppress checks="ClassFanOutComplexity"
|
||||
files="(KafkaStreams|KStreamImpl|KTableImpl|InternalTopologyBuilder|StreamsPartitionAssignor|StreamThread|IQv2StoreIntegrationTest).java"/>
|
||||
files="(KafkaStreams|KStreamImpl|KTableImpl|InternalTopologyBuilder|StreamsPartitionAssignor|StreamThread|IQv2StoreIntegrationTest|KStreamImplTest).java"/>
|
||||
|
||||
<suppress checks="MethodLength"
|
||||
files="KTableImpl.java"/>
|
||||
|
|
|
@ -119,6 +119,7 @@ public final class WordCountTransformerDemo {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
public static void main(final String[] args) throws IOException {
|
||||
final Properties props = new Properties();
|
||||
if (args != null && args.length > 0) {
|
||||
|
|
|
@ -25,6 +25,9 @@ import org.apache.kafka.streams.StreamsBuilder;
|
|||
import org.apache.kafka.streams.StreamsConfig;
|
||||
import org.apache.kafka.streams.Topology;
|
||||
import org.apache.kafka.streams.processor.ConnectedStoreProvider;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessor;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessorContext;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.api.Processor;
|
||||
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.api.ProcessorContext;
|
||||
|
@ -3205,7 +3208,9 @@ public interface KStream<K, V> {
|
|||
* @see #transformValues(ValueTransformerSupplier, String...)
|
||||
* @see #transformValues(ValueTransformerWithKeySupplier, String...)
|
||||
* @see #process(ProcessorSupplier, String...)
|
||||
* @deprecated Since 3.3. Use {@link KStream#process(ProcessorSupplier, String...)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
<K1, V1> KStream<K1, V1> transform(final TransformerSupplier<? super K, ? super V, KeyValue<K1, V1>> transformerSupplier,
|
||||
final String... stateStoreNames);
|
||||
|
||||
|
@ -3334,7 +3339,9 @@ public interface KStream<K, V> {
|
|||
* @see #transformValues(ValueTransformerSupplier, String...)
|
||||
* @see #transformValues(ValueTransformerWithKeySupplier, String...)
|
||||
* @see #process(ProcessorSupplier, String...)
|
||||
* @deprecated Since 3.3. Use {@link KStream#process(ProcessorSupplier, Named, String...)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
<K1, V1> KStream<K1, V1> transform(final TransformerSupplier<? super K, ? super V, KeyValue<K1, V1>> transformerSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames);
|
||||
|
@ -3463,7 +3470,9 @@ public interface KStream<K, V> {
|
|||
* @see #transformValues(ValueTransformerSupplier, String...)
|
||||
* @see #transformValues(ValueTransformerWithKeySupplier, String...)
|
||||
* @see #process(ProcessorSupplier, String...)
|
||||
* @deprecated Since 3.3. Use {@link KStream#process(ProcessorSupplier, String...)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
<K1, V1> KStream<K1, V1> flatTransform(final TransformerSupplier<? super K, ? super V, Iterable<KeyValue<K1, V1>>> transformerSupplier,
|
||||
final String... stateStoreNames);
|
||||
|
||||
|
@ -3592,7 +3601,9 @@ public interface KStream<K, V> {
|
|||
* @see #transformValues(ValueTransformerSupplier, String...)
|
||||
* @see #transformValues(ValueTransformerWithKeySupplier, String...)
|
||||
* @see #process(ProcessorSupplier, String...)
|
||||
* @deprecated Since 3.3. Use {@link KStream#process(ProcessorSupplier, Named, String...)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
<K1, V1> KStream<K1, V1> flatTransform(final TransformerSupplier<? super K, ? super V, Iterable<KeyValue<K1, V1>>> transformerSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames);
|
||||
|
@ -3702,7 +3713,9 @@ public interface KStream<K, V> {
|
|||
* @see #mapValues(ValueMapper)
|
||||
* @see #mapValues(ValueMapperWithKey)
|
||||
* @see #transform(TransformerSupplier, String...)
|
||||
* @deprecated Since 3.3. Use {@link KStream#processValues(FixedKeyProcessorSupplier, String...)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
<VR> KStream<K, VR> transformValues(final ValueTransformerSupplier<? super V, ? extends VR> valueTransformerSupplier,
|
||||
final String... stateStoreNames);
|
||||
/**
|
||||
|
@ -3812,7 +3825,9 @@ public interface KStream<K, V> {
|
|||
* @see #mapValues(ValueMapper)
|
||||
* @see #mapValues(ValueMapperWithKey)
|
||||
* @see #transform(TransformerSupplier, String...)
|
||||
* @deprecated Since 3.3. Use {@link KStream#processValues(FixedKeyProcessorSupplier, Named, String...)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
<VR> KStream<K, VR> transformValues(final ValueTransformerSupplier<? super V, ? extends VR> valueTransformerSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames);
|
||||
|
@ -3926,7 +3941,9 @@ public interface KStream<K, V> {
|
|||
* @see #mapValues(ValueMapper)
|
||||
* @see #mapValues(ValueMapperWithKey)
|
||||
* @see #transform(TransformerSupplier, String...)
|
||||
* @deprecated Since 3.3. Use {@link KStream#processValues(FixedKeyProcessorSupplier, String...)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
<VR> KStream<K, VR> transformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, ? extends VR> valueTransformerSupplier,
|
||||
final String... stateStoreNames);
|
||||
|
||||
|
@ -4040,7 +4057,9 @@ public interface KStream<K, V> {
|
|||
* @see #mapValues(ValueMapper)
|
||||
* @see #mapValues(ValueMapperWithKey)
|
||||
* @see #transform(TransformerSupplier, String...)
|
||||
* @deprecated Since 3.3. Use {@link KStream#processValues(FixedKeyProcessorSupplier, Named, String...)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
<VR> KStream<K, VR> transformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, ? extends VR> valueTransformerSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames);
|
||||
|
@ -4163,7 +4182,9 @@ public interface KStream<K, V> {
|
|||
* @see #mapValues(ValueMapperWithKey)
|
||||
* @see #transform(TransformerSupplier, String...)
|
||||
* @see #flatTransform(TransformerSupplier, String...)
|
||||
* @deprecated Since 3.3. Use {@link KStream#processValues(FixedKeyProcessorSupplier, String...)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
<VR> KStream<K, VR> flatTransformValues(final ValueTransformerSupplier<? super V, Iterable<VR>> valueTransformerSupplier,
|
||||
final String... stateStoreNames);
|
||||
|
||||
|
@ -4287,7 +4308,9 @@ public interface KStream<K, V> {
|
|||
* @see #mapValues(ValueMapperWithKey)
|
||||
* @see #transform(TransformerSupplier, String...)
|
||||
* @see #flatTransform(TransformerSupplier, String...)
|
||||
* @deprecated Since 3.3. Use {@link KStream#processValues(FixedKeyProcessorSupplier, Named, String...)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
<VR> KStream<K, VR> flatTransformValues(final ValueTransformerSupplier<? super V, Iterable<VR>> valueTransformerSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames);
|
||||
|
@ -4412,7 +4435,9 @@ public interface KStream<K, V> {
|
|||
* @see #mapValues(ValueMapperWithKey)
|
||||
* @see #transform(TransformerSupplier, String...)
|
||||
* @see #flatTransform(TransformerSupplier, String...)
|
||||
* @deprecated Since 3.3. Use {@link KStream#processValues(FixedKeyProcessorSupplier, String...)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
<VR> KStream<K, VR> flatTransformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, Iterable<VR>> valueTransformerSupplier,
|
||||
final String... stateStoreNames);
|
||||
|
||||
|
@ -4537,7 +4562,9 @@ public interface KStream<K, V> {
|
|||
* @see #mapValues(ValueMapperWithKey)
|
||||
* @see #transform(TransformerSupplier, String...)
|
||||
* @see #flatTransform(TransformerSupplier, String...)
|
||||
* @deprecated Since 3.3. Use {@link KStream#processValues(FixedKeyProcessorSupplier, Named, String...)} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
<VR> KStream<K, VR> flatTransformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, Iterable<VR>> valueTransformerSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames);
|
||||
|
@ -4644,101 +4671,6 @@ public interface KStream<K, V> {
|
|||
void process(final org.apache.kafka.streams.processor.ProcessorSupplier<? super K, ? super V> processorSupplier,
|
||||
final String... stateStoreNames);
|
||||
|
||||
|
||||
/**
|
||||
* Process all records in this stream, one record at a time, by applying a {@link Processor} (provided by the given
|
||||
* {@link ProcessorSupplier}).
|
||||
* 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.
|
||||
* <p>
|
||||
* In order for the processor to use state stores, the stores must be added to the topology and connected to the
|
||||
* processor using at least one of two strategies (though it's not required to connect global state stores; read-only
|
||||
* access to global state stores is available by default).
|
||||
* <p>
|
||||
* The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)},
|
||||
* and specify the store names via {@code stateStoreNames} so they will be connected to the processor.
|
||||
* <pre>{@code
|
||||
* // create store
|
||||
* StoreBuilder<KeyValueStore<String,String>> keyValueStoreBuilder =
|
||||
* Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("myProcessorState"),
|
||||
* Serdes.String(),
|
||||
* Serdes.String());
|
||||
* // add store
|
||||
* builder.addStateStore(keyValueStoreBuilder);
|
||||
*
|
||||
* KStream outputStream = inputStream.processor(new ProcessorSupplier() {
|
||||
* public Processor get() {
|
||||
* return new MyProcessor();
|
||||
* }
|
||||
* }, "myProcessorState");
|
||||
* }</pre>
|
||||
* The second strategy is for the given {@link ProcessorSupplier} to implement {@link ConnectedStoreProvider#stores()},
|
||||
* which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the processor.
|
||||
* <pre>{@code
|
||||
* class MyProcessorSupplier implements ProcessorSupplier {
|
||||
* // supply processor
|
||||
* Processor get() {
|
||||
* return new MyProcessor();
|
||||
* }
|
||||
*
|
||||
* // provide store(s) that will be added and connected to the associated processor
|
||||
* // the store name from the builder ("myProcessorState") is used to access the store later via the ProcessorContext
|
||||
* Set<StoreBuilder> stores() {
|
||||
* StoreBuilder<KeyValueStore<String, String>> keyValueStoreBuilder =
|
||||
* Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("myProcessorState"),
|
||||
* Serdes.String(),
|
||||
* Serdes.String());
|
||||
* return Collections.singleton(keyValueStoreBuilder);
|
||||
* }
|
||||
* }
|
||||
*
|
||||
* ...
|
||||
*
|
||||
* KStream outputStream = inputStream.process(new MyProcessorSupplier());
|
||||
* }</pre>
|
||||
* <p>
|
||||
* With either strategy, within the {@link Processor}, the state is obtained via the {@link ProcessorContext}.
|
||||
* To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()},
|
||||
* a schedule must be registered.
|
||||
* <pre>{@code
|
||||
* class MyProcessor implements Processor {
|
||||
* private StateStore state;
|
||||
*
|
||||
* void init(ProcessorContext context) {
|
||||
* this.state = context.getStateStore("myProcessorState");
|
||||
* // punctuate each second, can access this.state
|
||||
* context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, new Punctuator(..));
|
||||
* }
|
||||
*
|
||||
* void process(K key, V value) {
|
||||
* // can access this.state
|
||||
* }
|
||||
*
|
||||
* void close() {
|
||||
* // can access this.state
|
||||
* }
|
||||
* }
|
||||
* }</pre>
|
||||
* Even if any upstream operation was key-changing, no auto-repartition is triggered.
|
||||
* If repartitioning is required, a call to {@link #repartition()} should be performed before {@code process()}.
|
||||
*
|
||||
* @param processorSupplier an instance of {@link ProcessorSupplier} that generates a newly constructed {@link Processor}
|
||||
* The supplier should always generate a new instance. Creating a single {@link Processor} object
|
||||
* and returning the same object reference in {@link ProcessorSupplier#get()} is a
|
||||
* violation of the supplier pattern and leads to runtime exceptions.
|
||||
* @param stateStoreNames the names of the state stores used by the processor; not required if the supplier
|
||||
* implements {@link ConnectedStoreProvider#stores()}
|
||||
* @see #foreach(ForeachAction)
|
||||
* @see #transform(TransformerSupplier, String...)
|
||||
*/
|
||||
void process(final ProcessorSupplier<? super K, ? super V, Void, Void> processorSupplier,
|
||||
final String... stateStoreNames);
|
||||
|
||||
/**
|
||||
* Process all records in this stream, one record at a time, by applying a
|
||||
* {@link org.apache.kafka.streams.processor.Processor} (provided by the given
|
||||
|
@ -4845,13 +4777,13 @@ 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}).
|
||||
* 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.
|
||||
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #map(KeyValueMapper)}).
|
||||
* If you choose not to attach one, this operation is similar to the stateless {@link #map(KeyValueMapper)}
|
||||
* but allows access to the {@link org.apache.kafka.streams.processor.api.ProcessorContext}
|
||||
* and {@link org.apache.kafka.streams.processor.api.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.
|
||||
* <p>
|
||||
* In order for the processor to use state stores, the stores must be added to the topology and connected to the
|
||||
* processor using at least one of two strategies (though it's not required to connect global state stores; read-only
|
||||
|
@ -4868,7 +4800,7 @@ public interface KStream<K, V> {
|
|||
* // add store
|
||||
* builder.addStateStore(keyValueStoreBuilder);
|
||||
*
|
||||
* KStream outputStream = inputStream.processor(new ProcessorSupplier() {
|
||||
* KStream outputStream = inputStream.process(new ProcessorSupplier() {
|
||||
* public Processor get() {
|
||||
* return new MyProcessor();
|
||||
* }
|
||||
|
@ -4912,7 +4844,7 @@ public interface KStream<K, V> {
|
|||
* context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, new Punctuator(..));
|
||||
* }
|
||||
*
|
||||
* void process(K key, V value) {
|
||||
* void process(Record<K, V> record) {
|
||||
* // can access this.state
|
||||
* }
|
||||
*
|
||||
|
@ -4923,6 +4855,110 @@ public interface KStream<K, V> {
|
|||
* }</pre>
|
||||
* Even if any upstream operation was key-changing, no auto-repartition is triggered.
|
||||
* If repartitioning is required, a call to {@link #repartition()} should be performed before {@code process()}.
|
||||
* <p>
|
||||
* Processing records might result in an internal data redistribution if a key based operator (like an aggregation
|
||||
* or join) is applied to the result {@code KStream}.
|
||||
* (cf. {@link #processValues(FixedKeyProcessorSupplier, String...)})
|
||||
*
|
||||
* @param processorSupplier an instance of {@link ProcessorSupplier} that generates a newly constructed {@link Processor}
|
||||
* The supplier should always generate a new instance. Creating a single {@link Processor} object
|
||||
* and returning the same object reference in {@link ProcessorSupplier#get()} is a
|
||||
* violation of the supplier pattern and leads to runtime exceptions.
|
||||
* @param stateStoreNames the names of the state stores used by the processor; not required if the supplier
|
||||
* implements {@link ConnectedStoreProvider#stores()}
|
||||
* @see #map(KeyValueMapper)
|
||||
* @see #transform(TransformerSupplier, String...)
|
||||
*/
|
||||
<KOut, VOut> KStream<KOut, VOut> process(
|
||||
final ProcessorSupplier<? super K, ? super V, KOut, VOut> processorSupplier,
|
||||
final String... stateStoreNames
|
||||
);
|
||||
|
||||
/**
|
||||
* Process all records in this stream, one record at a time, by applying a {@link Processor} (provided by the given
|
||||
* {@link ProcessorSupplier}).
|
||||
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #map(KeyValueMapper)}).
|
||||
* If you choose not to attach one, this operation is similar to the stateless {@link #map(KeyValueMapper)}
|
||||
* but allows access to the {@link org.apache.kafka.streams.processor.api.ProcessorContext}
|
||||
* and {@link org.apache.kafka.streams.processor.api.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>
|
||||
* In order for the processor to use state stores, the stores must be added to the topology and connected to the
|
||||
* processor using at least one of two strategies (though it's not required to connect global state stores; read-only
|
||||
* access to global state stores is available by default).
|
||||
* <p>
|
||||
* The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)},
|
||||
* and specify the store names via {@code stateStoreNames} so they will be connected to the processor.
|
||||
* <pre>{@code
|
||||
* // create store
|
||||
* StoreBuilder<KeyValueStore<String,String>> keyValueStoreBuilder =
|
||||
* Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("myProcessorState"),
|
||||
* Serdes.String(),
|
||||
* Serdes.String());
|
||||
* // add store
|
||||
* builder.addStateStore(keyValueStoreBuilder);
|
||||
*
|
||||
* KStream outputStream = inputStream.process(new ProcessorSupplier() {
|
||||
* public Processor get() {
|
||||
* return new MyProcessor();
|
||||
* }
|
||||
* }, "myProcessorState");
|
||||
* }</pre>
|
||||
* The second strategy is for the given {@link ProcessorSupplier} to implement {@link ConnectedStoreProvider#stores()},
|
||||
* which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the processor.
|
||||
* <pre>{@code
|
||||
* class MyProcessorSupplier implements ProcessorSupplier {
|
||||
* // supply processor
|
||||
* Processor get() {
|
||||
* return new MyProcessor();
|
||||
* }
|
||||
*
|
||||
* // provide store(s) that will be added and connected to the associated processor
|
||||
* // the store name from the builder ("myProcessorState") is used to access the store later via the ProcessorContext
|
||||
* Set<StoreBuilder> stores() {
|
||||
* StoreBuilder<KeyValueStore<String, String>> keyValueStoreBuilder =
|
||||
* Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("myProcessorState"),
|
||||
* Serdes.String(),
|
||||
* Serdes.String());
|
||||
* return Collections.singleton(keyValueStoreBuilder);
|
||||
* }
|
||||
* }
|
||||
*
|
||||
* ...
|
||||
*
|
||||
* KStream outputStream = inputStream.process(new MyProcessorSupplier());
|
||||
* }</pre>
|
||||
* <p>
|
||||
* With either strategy, within the {@link Processor}, the state is obtained via the {@link ProcessorContext}.
|
||||
* To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()},
|
||||
* a schedule must be registered.
|
||||
* <pre>{@code
|
||||
* class MyProcessor implements Processor {
|
||||
* private StateStore state;
|
||||
*
|
||||
* void init(ProcessorContext context) {
|
||||
* this.state = context.getStateStore("myProcessorState");
|
||||
* // punctuate each second, can access this.state
|
||||
* context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, new Punctuator(..));
|
||||
* }
|
||||
*
|
||||
* void process(Record<K, V> record) {
|
||||
* // can access this.state
|
||||
* }
|
||||
*
|
||||
* void close() {
|
||||
* // can access this.state
|
||||
* }
|
||||
* }
|
||||
* }</pre>
|
||||
* Even if any upstream operation was key-changing, no auto-repartition is triggered.
|
||||
* If repartitioning is required, a call to {@link #repartition()} should be performed before {@code process()}.
|
||||
* <p>
|
||||
* Processing records might result in an internal data redistribution if a key based operator (like an aggregation
|
||||
* or join) is applied to the result {@code KStream}.
|
||||
* (cf. {@link #processValues(FixedKeyProcessorSupplier, Named, String...)})
|
||||
*
|
||||
* @param processorSupplier an instance of {@link ProcessorSupplier} that generates a newly constructed {@link Processor}
|
||||
* The supplier should always generate a new instance. Creating a single {@link Processor} object
|
||||
|
@ -4930,10 +4966,212 @@ public interface KStream<K, V> {
|
|||
* violation of the supplier pattern and leads to runtime exceptions.
|
||||
* @param named a {@link Named} config used to name the processor in the topology
|
||||
* @param stateStoreNames the names of the state store used by the processor
|
||||
* @see #foreach(ForeachAction)
|
||||
* @see #transform(TransformerSupplier, String...)
|
||||
* @see #map(KeyValueMapper)
|
||||
* @see #processValues(FixedKeyProcessorSupplier, Named, String...)
|
||||
*/
|
||||
void process(final ProcessorSupplier<? super K, ? super V, Void, Void> processorSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames);
|
||||
<KOut, VOut> KStream<KOut, VOut> process(
|
||||
final ProcessorSupplier<? super K, ? super V, KOut, VOut> processorSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames
|
||||
);
|
||||
|
||||
/**
|
||||
* Process all records in this stream, one record at a time, by applying a {@link FixedKeyProcessor} (provided by the given
|
||||
* {@link FixedKeyProcessorSupplier}).
|
||||
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper)}).
|
||||
* If you choose not to attach one, this operation is similar to the stateless {@link #mapValues(ValueMapper)}
|
||||
* but allows access to the {@link org.apache.kafka.streams.processor.api.ProcessorContext}
|
||||
* and {@link org.apache.kafka.streams.processor.api.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>
|
||||
* In order for the processor to use state stores, the stores must be added to the topology and connected to the
|
||||
* processor using at least one of two strategies (though it's not required to connect global state stores; read-only
|
||||
* access to global state stores is available by default).
|
||||
* <p>
|
||||
* The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)},
|
||||
* and specify the store names via {@code stateStoreNames} so they will be connected to the processor.
|
||||
* <pre>{@code
|
||||
* // create store
|
||||
* StoreBuilder<KeyValueStore<String,String>> keyValueStoreBuilder =
|
||||
* Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("myProcessorState"),
|
||||
* Serdes.String(),
|
||||
* Serdes.String());
|
||||
* // add store
|
||||
* builder.addStateStore(keyValueStoreBuilder);
|
||||
*
|
||||
* KStream outputStream = inputStream.processValues(new ProcessorSupplier() {
|
||||
* public Processor get() {
|
||||
* return new MyProcessor();
|
||||
* }
|
||||
* }, "myProcessorState");
|
||||
* }</pre>
|
||||
* The second strategy is for the given {@link ProcessorSupplier} to implement {@link ConnectedStoreProvider#stores()},
|
||||
* which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the processor.
|
||||
* <pre>{@code
|
||||
* class MyProcessorSupplier implements FixedKeyProcessorSupplier {
|
||||
* // supply processor
|
||||
* FixedKeyProcessor get() {
|
||||
* return new MyProcessor();
|
||||
* }
|
||||
*
|
||||
* // provide store(s) that will be added and connected to the associated processor
|
||||
* // the store name from the builder ("myProcessorState") is used to access the store later via the ProcessorContext
|
||||
* Set<StoreBuilder> stores() {
|
||||
* StoreBuilder<KeyValueStore<String, String>> keyValueStoreBuilder =
|
||||
* Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("myProcessorState"),
|
||||
* Serdes.String(),
|
||||
* Serdes.String());
|
||||
* return Collections.singleton(keyValueStoreBuilder);
|
||||
* }
|
||||
* }
|
||||
*
|
||||
* ...
|
||||
*
|
||||
* KStream outputStream = inputStream.processValues(new MyProcessorSupplier());
|
||||
* }</pre>
|
||||
* <p>
|
||||
* With either strategy, within the {@link FixedKeyProcessor}, the state is obtained via the {@link FixedKeyProcessorContext}.
|
||||
* To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()},
|
||||
* a schedule must be registered.
|
||||
* <pre>{@code
|
||||
* class MyProcessor implements FixedKeyProcessor {
|
||||
* private StateStore state;
|
||||
*
|
||||
* void init(ProcessorContext context) {
|
||||
* this.state = context.getStateStore("myProcessorState");
|
||||
* // punctuate each second, can access this.state
|
||||
* context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, new Punctuator(..));
|
||||
* }
|
||||
*
|
||||
* void process(FixedKeyRecord<K, V> record) {
|
||||
* // can access this.state
|
||||
* }
|
||||
*
|
||||
* void close() {
|
||||
* // can access this.state
|
||||
* }
|
||||
* }
|
||||
* }</pre>
|
||||
* Even if any upstream operation was key-changing, no auto-repartition is triggered.
|
||||
* If repartitioning is required, a call to {@link #repartition()} should be performed before {@code process()}.
|
||||
* <p>
|
||||
* Setting a new value preserves data co-location with respect to the key.
|
||||
* Thus, <em>no</em> internal data redistribution is required if a key based operator (like an aggregation or join)
|
||||
* is applied to the result {@code KStream}. (cf. {@link #process(ProcessorSupplier, String...)})
|
||||
*
|
||||
* @param processorSupplier an instance of {@link FixedKeyProcessorSupplier} that generates a newly constructed {@link FixedKeyProcessor}
|
||||
* The supplier should always generate a new instance. Creating a single {@link FixedKeyProcessor} object
|
||||
* and returning the same object reference in {@link FixedKeyProcessorSupplier#get()} is a
|
||||
* violation of the supplier pattern and leads to runtime exceptions.
|
||||
* @param stateStoreNames the names of the state store used by the processor
|
||||
* @see #mapValues(ValueMapper)
|
||||
* @see #process(ProcessorSupplier, Named, String...)
|
||||
*/
|
||||
<VOut> KStream<K, VOut> processValues(
|
||||
final FixedKeyProcessorSupplier<? super K, ? super V, VOut> processorSupplier,
|
||||
final String... stateStoreNames
|
||||
);
|
||||
|
||||
/**
|
||||
* Process all records in this stream, one record at a time, by applying a {@link FixedKeyProcessor} (provided by the given
|
||||
* {@link FixedKeyProcessorSupplier}).
|
||||
* Attaching a state store makes this a stateful record-by-record operation (cf. {@link #mapValues(ValueMapper)}).
|
||||
* If you choose not to attach one, this operation is similar to the stateless {@link #mapValues(ValueMapper)}
|
||||
* but allows access to the {@link org.apache.kafka.streams.processor.api.ProcessorContext}
|
||||
* and {@link org.apache.kafka.streams.processor.api.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>
|
||||
* In order for the processor to use state stores, the stores must be added to the topology and connected to the
|
||||
* processor using at least one of two strategies (though it's not required to connect global state stores; read-only
|
||||
* access to global state stores is available by default).
|
||||
* <p>
|
||||
* The first strategy is to manually add the {@link StoreBuilder}s via {@link Topology#addStateStore(StoreBuilder, String...)},
|
||||
* and specify the store names via {@code stateStoreNames} so they will be connected to the processor.
|
||||
* <pre>{@code
|
||||
* // create store
|
||||
* StoreBuilder<KeyValueStore<String,String>> keyValueStoreBuilder =
|
||||
* Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("myProcessorState"),
|
||||
* Serdes.String(),
|
||||
* Serdes.String());
|
||||
* // add store
|
||||
* builder.addStateStore(keyValueStoreBuilder);
|
||||
*
|
||||
* KStream outputStream = inputStream.processValues(new ProcessorSupplier() {
|
||||
* public Processor get() {
|
||||
* return new MyProcessor();
|
||||
* }
|
||||
* }, "myProcessorState");
|
||||
* }</pre>
|
||||
* The second strategy is for the given {@link ProcessorSupplier} to implement {@link ConnectedStoreProvider#stores()},
|
||||
* which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the processor.
|
||||
* <pre>{@code
|
||||
* class MyProcessorSupplier implements FixedKeyProcessorSupplier {
|
||||
* // supply processor
|
||||
* FixedKeyProcessor get() {
|
||||
* return new MyProcessor();
|
||||
* }
|
||||
*
|
||||
* // provide store(s) that will be added and connected to the associated processor
|
||||
* // the store name from the builder ("myProcessorState") is used to access the store later via the ProcessorContext
|
||||
* Set<StoreBuilder> stores() {
|
||||
* StoreBuilder<KeyValueStore<String, String>> keyValueStoreBuilder =
|
||||
* Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("myProcessorState"),
|
||||
* Serdes.String(),
|
||||
* Serdes.String());
|
||||
* return Collections.singleton(keyValueStoreBuilder);
|
||||
* }
|
||||
* }
|
||||
*
|
||||
* ...
|
||||
*
|
||||
* KStream outputStream = inputStream.processValues(new MyProcessorSupplier());
|
||||
* }</pre>
|
||||
* <p>
|
||||
* With either strategy, within the {@link FixedKeyProcessor}, the state is obtained via the {@link FixedKeyProcessorContext}.
|
||||
* To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()},
|
||||
* a schedule must be registered.
|
||||
* <pre>{@code
|
||||
* class MyProcessor implements FixedKeyProcessor {
|
||||
* private StateStore state;
|
||||
*
|
||||
* void init(ProcessorContext context) {
|
||||
* this.state = context.getStateStore("myProcessorState");
|
||||
* // punctuate each second, can access this.state
|
||||
* context.schedule(Duration.ofSeconds(1), PunctuationType.WALL_CLOCK_TIME, new Punctuator(..));
|
||||
* }
|
||||
*
|
||||
* void process(FixedKeyRecord<K, V> record) {
|
||||
* // can access this.state
|
||||
* }
|
||||
*
|
||||
* void close() {
|
||||
* // can access this.state
|
||||
* }
|
||||
* }
|
||||
* }</pre>
|
||||
* Even if any upstream operation was key-changing, no auto-repartition is triggered.
|
||||
* If repartitioning is required, a call to {@link #repartition()} should be performed before {@code process()}.
|
||||
* <p>
|
||||
* Setting a new value preserves data co-location with respect to the key.
|
||||
* Thus, <em>no</em> internal data redistribution is required if a key based operator (like an aggregation or join)
|
||||
* is applied to the result {@code KStream}. (cf. {@link #process(ProcessorSupplier, String...)})
|
||||
*
|
||||
* @param processorSupplier an instance of {@link FixedKeyProcessorSupplier} that generates a newly constructed {@link FixedKeyProcessor}
|
||||
* The supplier should always generate a new instance. Creating a single {@link FixedKeyProcessor} object
|
||||
* and returning the same object reference in {@link FixedKeyProcessorSupplier#get()} is a
|
||||
* violation of the supplier pattern and leads to runtime exceptions.
|
||||
* @param named a {@link Named} config used to name the processor in the topology
|
||||
* @param stateStoreNames the names of the state store used by the processor
|
||||
* @see #mapValues(ValueMapper)
|
||||
* @see #process(ProcessorSupplier, Named, String...)
|
||||
*/
|
||||
<VOut> KStream<K, VOut> processValues(
|
||||
final FixedKeyProcessorSupplier<? super K, ? super V, VOut> processorSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames
|
||||
);
|
||||
}
|
||||
|
|
|
@ -328,21 +328,27 @@ public class InternalStreamsBuilder implements InternalNameProvider {
|
|||
root.removeChild(graphNode);
|
||||
}
|
||||
} else {
|
||||
for (final String topic : currentSourceNode.topicNames().get()) {
|
||||
if (!topicsToSourceNodes.containsKey(topic)) {
|
||||
topicsToSourceNodes.put(topic, currentSourceNode);
|
||||
} else {
|
||||
final StreamSourceNode<?, ?> mainSourceNode = topicsToSourceNodes.get(topic);
|
||||
// TODO we only merge source nodes if the subscribed topic(s) are an exact match, so it's still not
|
||||
// possible to subscribe to topicA in one KStream and topicA + topicB in another. We could achieve
|
||||
// this by splitting these source nodes into one topic per node and routing to the subscribed children
|
||||
if (!mainSourceNode.topicNames().equals(currentSourceNode.topicNames())) {
|
||||
LOG.error("Topic {} was found in subscription for non-equal source nodes {} and {}",
|
||||
topic, mainSourceNode, currentSourceNode);
|
||||
throw new TopologyException("Two source nodes are subscribed to overlapping but not equal input topics");
|
||||
if (currentSourceNode.topicNames().isPresent()) {
|
||||
for (final String topic : currentSourceNode.topicNames().get()) {
|
||||
if (!topicsToSourceNodes.containsKey(topic)) {
|
||||
topicsToSourceNodes.put(topic, currentSourceNode);
|
||||
} else {
|
||||
final StreamSourceNode<?, ?> mainSourceNode = topicsToSourceNodes.get(
|
||||
topic);
|
||||
// TODO we only merge source nodes if the subscribed topic(s) are an exact match, so it's still not
|
||||
// possible to subscribe to topicA in one KStream and topicA + topicB in another. We could achieve
|
||||
// this by splitting these source nodes into one topic per node and routing to the subscribed children
|
||||
if (!mainSourceNode.topicNames()
|
||||
.equals(currentSourceNode.topicNames())) {
|
||||
LOG.error(
|
||||
"Topic {} was found in subscription for non-equal source nodes {} and {}",
|
||||
topic, mainSourceNode, currentSourceNode);
|
||||
throw new TopologyException(
|
||||
"Two source nodes are subscribed to overlapping but not equal input topics");
|
||||
}
|
||||
mainSourceNode.merge(currentSourceNode);
|
||||
root.removeChild(graphNode);
|
||||
}
|
||||
mainSourceNode.merge(currentSourceNode);
|
||||
root.removeChild(graphNode);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,12 +17,12 @@
|
|||
package org.apache.kafka.streams.kstream.internals;
|
||||
|
||||
import org.apache.kafka.streams.kstream.Predicate;
|
||||
import org.apache.kafka.streams.processor.api.ContextualProcessor;
|
||||
import org.apache.kafka.streams.processor.api.Processor;
|
||||
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
import org.apache.kafka.streams.processor.api.ContextualFixedKeyProcessor;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessor;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyRecord;
|
||||
|
||||
class KStreamFilter<K, V> implements ProcessorSupplier<K, V, K, V> {
|
||||
class KStreamFilter<K, V> implements FixedKeyProcessorSupplier<K, V, V> {
|
||||
|
||||
private final Predicate<K, V> predicate;
|
||||
private final boolean filterNot;
|
||||
|
@ -33,13 +33,13 @@ class KStreamFilter<K, V> implements ProcessorSupplier<K, V, K, V> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Processor<K, V, K, V> get() {
|
||||
public FixedKeyProcessor<K, V, V> get() {
|
||||
return new KStreamFilterProcessor();
|
||||
}
|
||||
|
||||
private class KStreamFilterProcessor extends ContextualProcessor<K, V, K, V> {
|
||||
private class KStreamFilterProcessor extends ContextualFixedKeyProcessor<K, V, V> {
|
||||
@Override
|
||||
public void process(final Record<K, V> record) {
|
||||
public void process(final FixedKeyRecord<K, V> record) {
|
||||
if (filterNot ^ predicate.test(record.key(), record.value())) {
|
||||
context().forward(record);
|
||||
}
|
||||
|
|
|
@ -17,12 +17,12 @@
|
|||
package org.apache.kafka.streams.kstream.internals;
|
||||
|
||||
import org.apache.kafka.streams.kstream.ValueMapperWithKey;
|
||||
import org.apache.kafka.streams.processor.api.ContextualProcessor;
|
||||
import org.apache.kafka.streams.processor.api.Processor;
|
||||
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
import org.apache.kafka.streams.processor.api.ContextualFixedKeyProcessor;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessor;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyRecord;
|
||||
|
||||
class KStreamFlatMapValues<KIn, VIn, VOut> implements ProcessorSupplier<KIn, VIn, KIn, VOut> {
|
||||
class KStreamFlatMapValues<KIn, VIn, VOut> implements FixedKeyProcessorSupplier<KIn, VIn, VOut> {
|
||||
|
||||
private final ValueMapperWithKey<? super KIn, ? super VIn, ? extends Iterable<? extends VOut>> mapper;
|
||||
|
||||
|
@ -31,13 +31,14 @@ class KStreamFlatMapValues<KIn, VIn, VOut> implements ProcessorSupplier<KIn, VIn
|
|||
}
|
||||
|
||||
@Override
|
||||
public Processor<KIn, VIn, KIn, VOut> get() {
|
||||
public FixedKeyProcessor<KIn, VIn, VOut> get() {
|
||||
return new KStreamFlatMapValuesProcessor();
|
||||
}
|
||||
|
||||
private class KStreamFlatMapValuesProcessor extends ContextualProcessor<KIn, VIn, KIn, VOut> {
|
||||
private class KStreamFlatMapValuesProcessor extends
|
||||
ContextualFixedKeyProcessor<KIn, VIn, VOut> {
|
||||
@Override
|
||||
public void process(final Record<KIn, VIn> record) {
|
||||
public void process(final FixedKeyRecord<KIn, VIn> record) {
|
||||
final Iterable<? extends VOut> newValues = mapper.apply(record.key(), record.value());
|
||||
for (final VOut v : newValues) {
|
||||
context().forward(record.withValue(v));
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.kafka.streams.KeyValue;
|
|||
import org.apache.kafka.streams.internals.ApiUtils;
|
||||
import org.apache.kafka.streams.kstream.BranchedKStream;
|
||||
import org.apache.kafka.streams.kstream.ForeachAction;
|
||||
import org.apache.kafka.streams.kstream.ForeachProcessor;
|
||||
import org.apache.kafka.streams.kstream.GlobalKTable;
|
||||
import org.apache.kafka.streams.kstream.Grouped;
|
||||
import org.apache.kafka.streams.kstream.JoinWindows;
|
||||
|
@ -46,6 +47,7 @@ import org.apache.kafka.streams.kstream.ValueTransformerSupplier;
|
|||
import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier;
|
||||
import org.apache.kafka.streams.kstream.internals.graph.BaseRepartitionNode;
|
||||
import org.apache.kafka.streams.kstream.internals.graph.BaseRepartitionNode.BaseRepartitionNodeBuilder;
|
||||
import org.apache.kafka.streams.kstream.internals.graph.GraphNode;
|
||||
import org.apache.kafka.streams.kstream.internals.graph.OptimizableRepartitionNode;
|
||||
import org.apache.kafka.streams.kstream.internals.graph.OptimizableRepartitionNode.OptimizableRepartitionNodeBuilder;
|
||||
import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
|
||||
|
@ -54,14 +56,13 @@ import org.apache.kafka.streams.kstream.internals.graph.StatefulProcessorNode;
|
|||
import org.apache.kafka.streams.kstream.internals.graph.StreamSinkNode;
|
||||
import org.apache.kafka.streams.kstream.internals.graph.StreamTableJoinNode;
|
||||
import org.apache.kafka.streams.kstream.internals.graph.StreamToTableNode;
|
||||
import org.apache.kafka.streams.kstream.internals.graph.GraphNode;
|
||||
import org.apache.kafka.streams.kstream.internals.graph.UnoptimizableRepartitionNode;
|
||||
import org.apache.kafka.streams.kstream.internals.graph.UnoptimizableRepartitionNode.UnoptimizableRepartitionNodeBuilder;
|
||||
import org.apache.kafka.streams.processor.FailOnInvalidTimestamp;
|
||||
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.StreamPartitioner;
|
||||
import org.apache.kafka.streams.processor.TopicNameExtractor;
|
||||
import org.apache.kafka.streams.kstream.ForeachProcessor;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.internals.InternalTopicProperties;
|
||||
import org.apache.kafka.streams.processor.internals.StaticTopicNameExtractor;
|
||||
import org.apache.kafka.streams.state.KeyValueStore;
|
||||
|
@ -119,6 +120,8 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
|
||||
private static final String PROCESSOR_NAME = "KSTREAM-PROCESSOR-";
|
||||
|
||||
private static final String PROCESSVALUES_NAME = "KSTREAM-PROCESSVALUES-";
|
||||
|
||||
private static final String PRINTING_NAME = "KSTREAM-PRINTER-";
|
||||
|
||||
private static final String KEY_SELECT_NAME = "KSTREAM-KEY-SELECT-";
|
||||
|
@ -1278,6 +1281,7 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public <KR, VR> KStream<KR, VR> transform(final TransformerSupplier<? super K, ? super V, KeyValue<KR, VR>> transformerSupplier,
|
||||
final String... stateStoreNames) {
|
||||
Objects.requireNonNull(transformerSupplier, "transformerSupplier can't be null");
|
||||
|
@ -1286,6 +1290,7 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public <KR, VR> KStream<KR, VR> transform(final TransformerSupplier<? super K, ? super V, KeyValue<KR, VR>> transformerSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames) {
|
||||
|
@ -1294,6 +1299,7 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public <K1, V1> KStream<K1, V1> flatTransform(final TransformerSupplier<? super K, ? super V, Iterable<KeyValue<K1, V1>>> transformerSupplier,
|
||||
final String... stateStoreNames) {
|
||||
Objects.requireNonNull(transformerSupplier, "transformerSupplier can't be null");
|
||||
|
@ -1302,6 +1308,7 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public <K1, V1> KStream<K1, V1> flatTransform(final TransformerSupplier<? super K, ? super V, Iterable<KeyValue<K1, V1>>> transformerSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames) {
|
||||
|
@ -1334,6 +1341,7 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public <VR> KStream<K, VR> transformValues(final ValueTransformerSupplier<? super V, ? extends VR> valueTransformerSupplier,
|
||||
final String... stateStoreNames) {
|
||||
Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null");
|
||||
|
@ -1344,6 +1352,7 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public <VR> KStream<K, VR> transformValues(final ValueTransformerSupplier<? super V, ? extends VR> valueTransformerSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames) {
|
||||
|
@ -1356,6 +1365,7 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public <VR> KStream<K, VR> transformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, ? extends VR> valueTransformerSupplier,
|
||||
final String... stateStoreNames) {
|
||||
Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null");
|
||||
|
@ -1363,6 +1373,7 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public <VR> KStream<K, VR> transformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, ? extends VR> valueTransformerSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames) {
|
||||
|
@ -1401,6 +1412,7 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public <VR> KStream<K, VR> flatTransformValues(final ValueTransformerSupplier<? super V, Iterable<VR>> valueTransformerSupplier,
|
||||
final String... stateStoreNames) {
|
||||
Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null");
|
||||
|
@ -1411,6 +1423,7 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public <VR> KStream<K, VR> flatTransformValues(final ValueTransformerSupplier<? super V, Iterable<VR>> valueTransformerSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames) {
|
||||
|
@ -1422,6 +1435,7 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public <VR> KStream<K, VR> flatTransformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, Iterable<VR>> valueTransformerSupplier,
|
||||
final String... stateStoreNames) {
|
||||
Objects.requireNonNull(valueTransformerSupplier, "valueTransformerSupplier can't be null");
|
||||
|
@ -1429,6 +1443,7 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public <VR> KStream<K, VR> flatTransformValues(final ValueTransformerWithKeySupplier<? super K, ? super V, Iterable<VR>> valueTransformerSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames) {
|
||||
|
@ -1472,12 +1487,6 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
process(processorSupplier, Named.as(builder.newProcessorName(PROCESSOR_NAME)), stateStoreNames);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process(final ProcessorSupplier<? super K, ? super V, Void, Void> processorSupplier,
|
||||
final String... stateStoreNames) {
|
||||
process(processorSupplier, Named.as(builder.newProcessorName(PROCESSOR_NAME)), stateStoreNames);
|
||||
}
|
||||
|
||||
@Override
|
||||
@Deprecated
|
||||
public void process(final org.apache.kafka.streams.processor.ProcessorSupplier<? super K, ? super V> processorSupplier,
|
||||
|
@ -1501,9 +1510,23 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
}
|
||||
|
||||
@Override
|
||||
public void process(final ProcessorSupplier<? super K, ? super V, Void, Void> processorSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames) {
|
||||
public <KOut, VOut> KStream<KOut, VOut> process(
|
||||
final ProcessorSupplier<? super K, ? super V, KOut, VOut> processorSupplier,
|
||||
final String... stateStoreNames
|
||||
) {
|
||||
return process(
|
||||
processorSupplier,
|
||||
Named.as(builder.newProcessorName(PROCESSOR_NAME)),
|
||||
stateStoreNames
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <KOut, VOut> KStream<KOut, VOut> process(
|
||||
final ProcessorSupplier<? super K, ? super V, KOut, VOut> processorSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames
|
||||
) {
|
||||
Objects.requireNonNull(processorSupplier, "processorSupplier can't be null");
|
||||
Objects.requireNonNull(named, "named can't be null");
|
||||
Objects.requireNonNull(stateStoreNames, "stateStoreNames can't be a null array");
|
||||
|
@ -1519,5 +1542,59 @@ public class KStreamImpl<K, V> extends AbstractStream<K, V> implements KStream<K
|
|||
stateStoreNames);
|
||||
|
||||
builder.addGraphNode(graphNode, processNode);
|
||||
|
||||
// cannot inherit key and value serde
|
||||
return new KStreamImpl<>(
|
||||
name,
|
||||
null,
|
||||
null,
|
||||
subTopologySourceNodes,
|
||||
true,
|
||||
processNode,
|
||||
builder);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <VOut> KStream<K, VOut> processValues(
|
||||
final FixedKeyProcessorSupplier<? super K, ? super V, VOut> processorSupplier,
|
||||
final String... stateStoreNames
|
||||
) {
|
||||
return processValues(
|
||||
processorSupplier,
|
||||
Named.as(builder.newProcessorName(PROCESSVALUES_NAME)),
|
||||
stateStoreNames
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <VOut> KStream<K, VOut> processValues(
|
||||
final FixedKeyProcessorSupplier<? super K, ? super V, VOut> processorSupplier,
|
||||
final Named named,
|
||||
final String... stateStoreNames
|
||||
) {
|
||||
Objects.requireNonNull(processorSupplier, "processorSupplier can't be null");
|
||||
Objects.requireNonNull(named, "named can't be null");
|
||||
Objects.requireNonNull(stateStoreNames, "stateStoreNames can't be a null array");
|
||||
ApiUtils.checkSupplier(processorSupplier);
|
||||
for (final String stateStoreName : stateStoreNames) {
|
||||
Objects.requireNonNull(stateStoreName, "stateStoreNames can't be null");
|
||||
}
|
||||
|
||||
final String name = new NamedInternal(named).name();
|
||||
final StatefulProcessorNode<? super K, ? super V> processNode = new StatefulProcessorNode<>(
|
||||
name,
|
||||
new ProcessorParameters<>(processorSupplier, name),
|
||||
stateStoreNames);
|
||||
|
||||
builder.addGraphNode(graphNode, processNode);
|
||||
// cannot inherit value serde
|
||||
return new KStreamImpl<>(
|
||||
name,
|
||||
keySerde,
|
||||
null,
|
||||
subTopologySourceNodes,
|
||||
repartitionRequired,
|
||||
processNode,
|
||||
builder);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,12 +17,12 @@
|
|||
package org.apache.kafka.streams.kstream.internals;
|
||||
|
||||
import org.apache.kafka.streams.kstream.ValueMapperWithKey;
|
||||
import org.apache.kafka.streams.processor.api.ContextualProcessor;
|
||||
import org.apache.kafka.streams.processor.api.Processor;
|
||||
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
import org.apache.kafka.streams.processor.api.ContextualFixedKeyProcessor;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessor;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyRecord;
|
||||
|
||||
class KStreamMapValues<KIn, VIn, VOut> implements ProcessorSupplier<KIn, VIn, KIn, VOut> {
|
||||
class KStreamMapValues<KIn, VIn, VOut> implements FixedKeyProcessorSupplier<KIn, VIn, VOut> {
|
||||
|
||||
private final ValueMapperWithKey<KIn, VIn, VOut> mapper;
|
||||
|
||||
|
@ -31,13 +31,13 @@ class KStreamMapValues<KIn, VIn, VOut> implements ProcessorSupplier<KIn, VIn, KI
|
|||
}
|
||||
|
||||
@Override
|
||||
public Processor<KIn, VIn, KIn, VOut> get() {
|
||||
public FixedKeyProcessor<KIn, VIn, VOut> get() {
|
||||
return new KStreamMapProcessor();
|
||||
}
|
||||
|
||||
private class KStreamMapProcessor extends ContextualProcessor<KIn, VIn, KIn, VOut> {
|
||||
private class KStreamMapProcessor extends ContextualFixedKeyProcessor<KIn, VIn, VOut> {
|
||||
@Override
|
||||
public void process(final Record<KIn, VIn> record) {
|
||||
public void process(final FixedKeyRecord<KIn, VIn> record) {
|
||||
final VOut newValue = mapper.apply(record.key(), record.value());
|
||||
context().forward(record.withValue(newValue));
|
||||
}
|
||||
|
|
|
@ -17,12 +17,12 @@
|
|||
package org.apache.kafka.streams.kstream.internals;
|
||||
|
||||
import org.apache.kafka.streams.kstream.ForeachAction;
|
||||
import org.apache.kafka.streams.processor.api.ContextualProcessor;
|
||||
import org.apache.kafka.streams.processor.api.Processor;
|
||||
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
import org.apache.kafka.streams.processor.api.ContextualFixedKeyProcessor;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessor;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyRecord;
|
||||
|
||||
class KStreamPeek<K, V> implements ProcessorSupplier<K, V, K, V> {
|
||||
class KStreamPeek<K, V> implements FixedKeyProcessorSupplier<K, V, V> {
|
||||
|
||||
private final ForeachAction<K, V> action;
|
||||
|
||||
|
@ -31,13 +31,13 @@ class KStreamPeek<K, V> implements ProcessorSupplier<K, V, K, V> {
|
|||
}
|
||||
|
||||
@Override
|
||||
public Processor<K, V, K, V> get() {
|
||||
public FixedKeyProcessor<K, V, V> get() {
|
||||
return new KStreamPeekProcessor();
|
||||
}
|
||||
|
||||
private class KStreamPeekProcessor extends ContextualProcessor<K, V, K, V> {
|
||||
private class KStreamPeekProcessor extends ContextualFixedKeyProcessor<K, V, V> {
|
||||
@Override
|
||||
public void process(final Record<K, V> record) {
|
||||
public void process(final FixedKeyRecord<K, V> record) {
|
||||
action.apply(record.key(), record.value());
|
||||
context().forward(record);
|
||||
}
|
||||
|
|
|
@ -67,11 +67,7 @@ public class OptimizableRepartitionNode<K, V> extends BaseRepartitionNode<K, V>
|
|||
public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
|
||||
topologyBuilder.addInternalTopic(repartitionTopic, internalTopicProperties);
|
||||
|
||||
topologyBuilder.addProcessor(
|
||||
processorParameters.processorName(),
|
||||
processorParameters.processorSupplier(),
|
||||
parentNodeNames()
|
||||
);
|
||||
processorParameters.addProcessorTo(topologyBuilder, parentNodeNames());
|
||||
|
||||
topologyBuilder.addSink(
|
||||
sinkName,
|
||||
|
|
|
@ -56,7 +56,6 @@ public class ProcessorGraphNode<K, V> extends GraphNode {
|
|||
|
||||
@Override
|
||||
public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
|
||||
|
||||
topologyBuilder.addProcessor(processorParameters.processorName(), processorParameters.processorSupplier(), parentNodeNames());
|
||||
processorParameters.addProcessorTo(topologyBuilder, parentNodeNames());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,15 +20,21 @@ package org.apache.kafka.streams.kstream.internals.graph;
|
|||
import org.apache.kafka.streams.kstream.internals.KTableKTableJoinMerger;
|
||||
import org.apache.kafka.streams.kstream.internals.KTableProcessorSupplier;
|
||||
import org.apache.kafka.streams.kstream.internals.KTableSource;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
|
||||
import org.apache.kafka.streams.processor.internals.ProcessorAdapter;
|
||||
import org.apache.kafka.streams.state.StoreBuilder;
|
||||
|
||||
/**
|
||||
* Class used to represent a {@link ProcessorSupplier} and the name
|
||||
* Class used to represent a {@link ProcessorSupplier} or {@link FixedKeyProcessorSupplier} and the name
|
||||
* used to register it with the {@link org.apache.kafka.streams.processor.internals.InternalTopologyBuilder}
|
||||
*
|
||||
* Used by the Join nodes as there are several parameters, this abstraction helps
|
||||
* keep the number of arguments more reasonable.
|
||||
*
|
||||
* @see ProcessorSupplier
|
||||
* @see FixedKeyProcessorSupplier
|
||||
*/
|
||||
public class ProcessorParameters<KIn, VIn, KOut, VOut> {
|
||||
|
||||
|
@ -37,6 +43,7 @@ public class ProcessorParameters<KIn, VIn, KOut, VOut> {
|
|||
@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
|
||||
private final org.apache.kafka.streams.processor.ProcessorSupplier<KIn, VIn> oldProcessorSupplier;
|
||||
private final ProcessorSupplier<KIn, VIn, KOut, VOut> processorSupplier;
|
||||
private final FixedKeyProcessorSupplier<KIn, VIn, VOut> fixedKeyProcessorSupplier;
|
||||
private final String processorName;
|
||||
|
||||
@SuppressWarnings("deprecation") // Old PAPI compatibility.
|
||||
|
@ -44,6 +51,7 @@ public class ProcessorParameters<KIn, VIn, KOut, VOut> {
|
|||
final String processorName) {
|
||||
oldProcessorSupplier = processorSupplier;
|
||||
this.processorSupplier = () -> ProcessorAdapter.adapt(processorSupplier.get());
|
||||
fixedKeyProcessorSupplier = null;
|
||||
this.processorName = processorName;
|
||||
}
|
||||
|
||||
|
@ -51,6 +59,15 @@ public class ProcessorParameters<KIn, VIn, KOut, VOut> {
|
|||
final String processorName) {
|
||||
oldProcessorSupplier = null;
|
||||
this.processorSupplier = processorSupplier;
|
||||
fixedKeyProcessorSupplier = null;
|
||||
this.processorName = processorName;
|
||||
}
|
||||
|
||||
public ProcessorParameters(final FixedKeyProcessorSupplier<KIn, VIn, VOut> processorSupplier,
|
||||
final String processorName) {
|
||||
oldProcessorSupplier = null;
|
||||
this.processorSupplier = null;
|
||||
fixedKeyProcessorSupplier = processorSupplier;
|
||||
this.processorName = processorName;
|
||||
}
|
||||
|
||||
|
@ -58,9 +75,36 @@ public class ProcessorParameters<KIn, VIn, KOut, VOut> {
|
|||
return processorSupplier;
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
|
||||
public org.apache.kafka.streams.processor.ProcessorSupplier<KIn, VIn> oldProcessorSupplier() {
|
||||
return oldProcessorSupplier;
|
||||
public FixedKeyProcessorSupplier<KIn, VIn, VOut> fixedKeyProcessorSupplier() {
|
||||
return fixedKeyProcessorSupplier;
|
||||
}
|
||||
|
||||
public void addProcessorTo(final InternalTopologyBuilder topologyBuilder, final String[] parentNodeNames) {
|
||||
if (processorSupplier != null) {
|
||||
topologyBuilder.addProcessor(processorName, processorSupplier, parentNodeNames);
|
||||
if (processorSupplier.stores() != null) {
|
||||
for (final StoreBuilder<?> storeBuilder : processorSupplier.stores()) {
|
||||
topologyBuilder.addStateStore(storeBuilder, processorName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (fixedKeyProcessorSupplier != null) {
|
||||
topologyBuilder.addProcessor(processorName, fixedKeyProcessorSupplier, parentNodeNames);
|
||||
if (fixedKeyProcessorSupplier.stores() != null) {
|
||||
for (final StoreBuilder<?> storeBuilder : fixedKeyProcessorSupplier.stores()) {
|
||||
topologyBuilder.addStateStore(storeBuilder, processorName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// temporary hack until KIP-478 is fully implemented
|
||||
// Old PAPI. Needs to be migrated.
|
||||
if (oldProcessorSupplier != null && oldProcessorSupplier.stores() != null) {
|
||||
for (final StoreBuilder<?> storeBuilder : oldProcessorSupplier.stores()) {
|
||||
topologyBuilder.addStateStore(storeBuilder, processorName);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
|
|
|
@ -17,7 +17,6 @@
|
|||
package org.apache.kafka.streams.kstream.internals.graph;
|
||||
|
||||
import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier;
|
||||
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder;
|
||||
import org.apache.kafka.streams.state.StoreBuilder;
|
||||
|
||||
|
@ -79,35 +78,14 @@ public class StatefulProcessorNode<K, V> extends ProcessorGraphNode<K, V> {
|
|||
|
||||
@Override
|
||||
public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
|
||||
|
||||
final String processorName = processorParameters().processorName();
|
||||
final ProcessorSupplier<K, V, ?, ?> processorSupplier = processorParameters().processorSupplier();
|
||||
|
||||
topologyBuilder.addProcessor(processorName, processorSupplier, parentNodeNames());
|
||||
processorParameters().addProcessorTo(topologyBuilder, parentNodeNames());
|
||||
|
||||
if (storeNames != null && storeNames.length > 0) {
|
||||
topologyBuilder.connectProcessorAndStateStores(processorName, storeNames);
|
||||
topologyBuilder.connectProcessorAndStateStores(processorParameters().processorName(), storeNames);
|
||||
}
|
||||
|
||||
if (storeBuilder != null) {
|
||||
topologyBuilder.addStateStore(storeBuilder, processorName);
|
||||
topologyBuilder.addStateStore(storeBuilder, processorParameters().processorName());
|
||||
}
|
||||
|
||||
if (processorSupplier.stores() != null) {
|
||||
for (final StoreBuilder<?> storeBuilder : processorSupplier.stores()) {
|
||||
topologyBuilder.addStateStore(storeBuilder, processorName);
|
||||
}
|
||||
}
|
||||
|
||||
// temporary hack until KIP-478 is fully implemented
|
||||
@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
|
||||
final org.apache.kafka.streams.processor.ProcessorSupplier<K, V> oldProcessorSupplier =
|
||||
processorParameters().oldProcessorSupplier();
|
||||
if (oldProcessorSupplier != null && oldProcessorSupplier.stores() != null) {
|
||||
for (final StoreBuilder<?> storeBuilder : oldProcessorSupplier.stores()) {
|
||||
topologyBuilder.addStateStore(storeBuilder, processorName);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -53,11 +53,7 @@ public class UnoptimizableRepartitionNode<K, V> extends BaseRepartitionNode<K, V
|
|||
public void writeToTopology(final InternalTopologyBuilder topologyBuilder) {
|
||||
topologyBuilder.addInternalTopic(repartitionTopic, internalTopicProperties);
|
||||
|
||||
topologyBuilder.addProcessor(
|
||||
processorParameters.processorName(),
|
||||
processorParameters.processorSupplier(),
|
||||
parentNodeNames()
|
||||
);
|
||||
processorParameters.addProcessorTo(topologyBuilder, parentNodeNames());
|
||||
|
||||
topologyBuilder.addSink(
|
||||
sinkName,
|
||||
|
|
|
@ -0,0 +1,47 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.streams.processor.api;
|
||||
|
||||
/**
|
||||
* An abstract implementation of {@link FixedKeyProcessor} that manages the
|
||||
* {@link FixedKeyProcessorContext} instance and provides default no-op
|
||||
* implementation of {@link #close()}.
|
||||
*
|
||||
* @param <KIn> the type of input keys
|
||||
* @param <VIn> the type of input values
|
||||
* @param <VOut> the type of output values
|
||||
*/
|
||||
public abstract class ContextualFixedKeyProcessor<KIn, VIn, VOut> implements FixedKeyProcessor<KIn, VIn, VOut> {
|
||||
|
||||
private FixedKeyProcessorContext<KIn, VOut> context;
|
||||
|
||||
protected ContextualFixedKeyProcessor() {}
|
||||
|
||||
@Override
|
||||
public void init(final FixedKeyProcessorContext<KIn, VOut> context) {
|
||||
this.context = context;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the processor's context set during {@link #init(FixedKeyProcessorContext) initialization}.
|
||||
*
|
||||
* @return the processor context; null only when called prior to {@link #init(FixedKeyProcessorContext) initialization}.
|
||||
*/
|
||||
protected final FixedKeyProcessorContext<KIn, VOut> context() {
|
||||
return context;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,62 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.streams.processor.api;
|
||||
|
||||
import org.apache.kafka.streams.processor.PunctuationType;
|
||||
import org.apache.kafka.streams.processor.Punctuator;
|
||||
import org.apache.kafka.streams.processor.StateStore;
|
||||
|
||||
import java.time.Duration;
|
||||
|
||||
/**
|
||||
* A processor of key-value pair records where keys are immutable.
|
||||
*
|
||||
* @param <KIn> the type of input keys
|
||||
* @param <VIn> the type of input values
|
||||
* @param <VOut> the type of output values
|
||||
*/
|
||||
public interface FixedKeyProcessor<KIn, VIn, VOut> {
|
||||
|
||||
/**
|
||||
* Initialize this processor with the given context. The framework ensures this is called once per processor when the topology
|
||||
* that contains it is initialized. When the framework is done with the processor, {@link #close()} will be called on it; the
|
||||
* framework may later re-use the processor by calling {@code #init()} again.
|
||||
* <p>
|
||||
* The provided {@link FixedKeyProcessorContext context} can be used to access topology and record metadata, to
|
||||
* {@link FixedKeyProcessorContext#schedule(Duration, PunctuationType, Punctuator) schedule} a method to be
|
||||
* {@link Punctuator#punctuate(long) called periodically} and to access attached {@link StateStore}s.
|
||||
*
|
||||
* @param context the context; may not be null
|
||||
*/
|
||||
default void init(final FixedKeyProcessorContext<KIn, VOut> context) {}
|
||||
|
||||
/**
|
||||
* Process the record. Note that record metadata is undefined in cases such as a forward call from a punctuator.
|
||||
*
|
||||
* @param record the record to process
|
||||
*/
|
||||
void process(FixedKeyRecord<KIn, VIn> record);
|
||||
|
||||
/**
|
||||
* Close this processor and clean up any resources. Be aware that {@code #close()} is called after an internal cleanup.
|
||||
* Thus, it is not possible to write anything to Kafka as underlying clients are already closed. The framework may
|
||||
* later re-use this processor by calling {@code #init()} on it again.
|
||||
* <p>
|
||||
* Note: Do not close any streams managed resources, like {@link StateStore}s here, as they are managed by the library.
|
||||
*/
|
||||
default void close() {}
|
||||
}
|
|
@ -0,0 +1,94 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.streams.processor.api;
|
||||
|
||||
/**
|
||||
* Processor context interface for {@link FixedKeyRecord}.
|
||||
*
|
||||
* @param <KForward> a bound on the types of keys that may be forwarded
|
||||
* @param <VForward> a bound on the types of values that may be forwarded
|
||||
*/
|
||||
public interface FixedKeyProcessorContext<KForward, VForward> extends ProcessingContext {
|
||||
|
||||
/**
|
||||
* Forward a record to all child processors.
|
||||
* <p>
|
||||
* Note that the forwarded {@link FixedKeyRecord} is shared between the parent and child
|
||||
* processors. And of course, the parent may forward the same object to multiple children,
|
||||
* and the child may forward it to grandchildren, etc. Therefore, you should be mindful
|
||||
* of mutability.
|
||||
* <p>
|
||||
* The {@link FixedKeyRecord} class itself is immutable (all the setter-style methods return an
|
||||
* independent copy of the instance). However, the value and headers referenced by
|
||||
* the Record may themselves be mutable.
|
||||
* <p>
|
||||
* Some programs may opt to make use of this mutability for high performance, in which case
|
||||
* the input record may be mutated and then forwarded by each {@link FixedKeyProcessor}. However,
|
||||
* most applications should instead favor safety.
|
||||
* <p>
|
||||
* Forwarding records safely simply means to make a copy of the record before you mutate it.
|
||||
* This is trivial when using the {@link FixedKeyRecord#withValue(Object)},
|
||||
* and {@link FixedKeyRecord#withTimestamp(long)} methods, as each of these methods make a copy of the
|
||||
* record as a matter of course. But a little extra care must be taken with headers, since
|
||||
* the {@link org.apache.kafka.common.header.Header} class is mutable. The easiest way to
|
||||
* safely handle headers is to use the {@link FixedKeyRecord} constructors to make a copy before
|
||||
* modifying headers.
|
||||
* <p>
|
||||
* In other words, this would be considered unsafe:
|
||||
* <code>
|
||||
* process(FixedKeyRecord inputRecord) {
|
||||
* inputRecord.headers().add(...);
|
||||
* context.forward(inputRecord);
|
||||
* }
|
||||
* </code>
|
||||
* This is unsafe because the parent, and potentially siblings, grandparents, etc.,
|
||||
* all will see this modification to their shared Headers reference. This is a violation
|
||||
* of causality and could lead to undefined behavior.
|
||||
* <p>
|
||||
* A safe usage would look like this:
|
||||
* <code>
|
||||
* process(FixedKeyRecord inputRecord) {
|
||||
* // makes a copy of the headers
|
||||
* FixedKeyRecord toForward = inputRecord.withHeaders(inputRecord.headers());
|
||||
* // Other options to create a safe copy are:
|
||||
* // * use any copy-on-write method, which makes a copy of all fields:
|
||||
* // toForward = inputRecord.withValue();
|
||||
* // * explicitly copy all fields:
|
||||
* // toForward = new FixedKeyRecord(inputRecord.key(), inputRecord.value(), inputRecord.timestamp(), inputRecord.headers());
|
||||
* // * create a fresh, empty Headers:
|
||||
* // toForward = new FixedKeyRecord(inputRecord.key(), inputRecord.value(), inputRecord.timestamp());
|
||||
* // * etc.
|
||||
*
|
||||
* // now, we are modifying our own independent copy of the headers.
|
||||
* toForward.headers().add(...);
|
||||
* context.forward(toForward);
|
||||
* }
|
||||
* </code>
|
||||
* @param record The record to forward to all children
|
||||
*/
|
||||
<K extends KForward, V extends VForward> void forward(FixedKeyRecord<K, V> record);
|
||||
|
||||
/**
|
||||
* Forward a record to the specified child processor.
|
||||
* See {@link FixedKeyProcessorContext#forward(FixedKeyRecord)} for considerations.
|
||||
*
|
||||
* @param record The record to forward
|
||||
* @param childName The name of the child processor to receive the record
|
||||
* @see FixedKeyProcessorContext#forward(FixedKeyRecord)
|
||||
*/
|
||||
<K extends KForward, V extends VForward> void forward(FixedKeyRecord<K, V> record, final String childName);
|
||||
}
|
|
@ -0,0 +1,48 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.streams.processor.api;
|
||||
|
||||
import org.apache.kafka.streams.processor.ConnectedStoreProvider;
|
||||
|
||||
import java.util.function.Supplier;
|
||||
|
||||
/**
|
||||
* A processor supplier that can create one or more {@link FixedKeyProcessor} instances.
|
||||
* <p>
|
||||
* The supplier should always generate a new instance each time {@link FixedKeyProcessorSupplier#get()} gets called. Creating
|
||||
* a single {@link FixedKeyProcessor} object and returning the same object reference in {@link FixedKeyProcessorSupplier#get()} would be
|
||||
* a violation of the supplier pattern and leads to runtime exceptions.
|
||||
*
|
||||
* @param <KIn> the type of input keys
|
||||
* @param <VIn> the type of input values
|
||||
* @param <VOut> the type of output values
|
||||
*/
|
||||
@FunctionalInterface
|
||||
public interface FixedKeyProcessorSupplier<KIn, VIn, VOut>
|
||||
extends ConnectedStoreProvider, Supplier<FixedKeyProcessor<KIn, VIn, VOut>> {
|
||||
|
||||
/**
|
||||
* Return a newly constructed {@link FixedKeyProcessor} instance.
|
||||
* The supplier should always generate a new instance each time {@code FixedKeyProcessorSupplier#get()} gets called.
|
||||
* <p>
|
||||
* Creating a single {@link FixedKeyProcessor} object and returning the same object reference in {@code FixedKeyProcessorSupplier#get()}
|
||||
* is a violation of the supplier pattern and leads to runtime exceptions.
|
||||
*
|
||||
* @return a new {@link FixedKeyProcessor} instance
|
||||
*/
|
||||
FixedKeyProcessor<KIn, VIn, VOut> get();
|
||||
}
|
|
@ -0,0 +1,157 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.streams.processor.api;
|
||||
|
||||
import org.apache.kafka.common.header.Headers;
|
||||
import org.apache.kafka.common.header.internals.RecordHeaders;
|
||||
import org.apache.kafka.streams.errors.StreamsException;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* A data class representing an incoming record with fixed key for processing in a {@link FixedKeyProcessor}
|
||||
* or a record to forward to downstream processors via {@link FixedKeyProcessorContext}.
|
||||
*
|
||||
* This class encapsulates all the data attributes of a record: the key and value, but
|
||||
* also the timestamp of the record and any record headers.
|
||||
* Though key is not allowed to be changes.
|
||||
*
|
||||
* This class is immutable, though the objects referenced in the attributes of this class
|
||||
* may themselves be mutable.
|
||||
*
|
||||
* @param <K> The type of the fixed key
|
||||
* @param <V> The type of the value
|
||||
*/
|
||||
public final class FixedKeyRecord<K, V> {
|
||||
|
||||
private final K key;
|
||||
private final V value;
|
||||
private final long timestamp;
|
||||
private final Headers headers;
|
||||
|
||||
/**
|
||||
* Package-private constructor. Users must not construct this class directly, but only
|
||||
* modify records they were handed by the framework.
|
||||
*/
|
||||
FixedKeyRecord(final K key, final V value, final long timestamp, final Headers headers) {
|
||||
this.key = key;
|
||||
this.value = value;
|
||||
if (timestamp < 0) {
|
||||
throw new StreamsException(
|
||||
"Malformed Record",
|
||||
new IllegalArgumentException("Timestamp may not be negative. Got: " + timestamp)
|
||||
);
|
||||
}
|
||||
this.timestamp = timestamp;
|
||||
this.headers = new RecordHeaders(headers);
|
||||
}
|
||||
|
||||
/**
|
||||
* The key of the record. May be null.
|
||||
*/
|
||||
public K key() {
|
||||
return key;
|
||||
}
|
||||
|
||||
/**
|
||||
* The value of the record. May be null.
|
||||
*/
|
||||
public V value() {
|
||||
return value;
|
||||
}
|
||||
|
||||
/**
|
||||
* The timestamp of the record. Will never be negative.
|
||||
*/
|
||||
public long timestamp() {
|
||||
return timestamp;
|
||||
}
|
||||
|
||||
/**
|
||||
* The headers of the record. Never null.
|
||||
*/
|
||||
public Headers headers() {
|
||||
return headers;
|
||||
}
|
||||
|
||||
/**
|
||||
* A convenient way to produce a new record if you only need to change the value.
|
||||
*
|
||||
* Copies the attributes of this record with the value replaced.
|
||||
*
|
||||
* @param value The value of the result record.
|
||||
* @param <NewV> The type of the new record's value.
|
||||
* @return A new Record instance with all the same attributes (except that the value is replaced).
|
||||
*/
|
||||
public <NewV> FixedKeyRecord<K, NewV> withValue(final NewV value) {
|
||||
return new FixedKeyRecord<>(key, value, timestamp, headers);
|
||||
}
|
||||
|
||||
/**
|
||||
* A convenient way to produce a new record if you only need to change the timestamp.
|
||||
*
|
||||
* Copies the attributes of this record with the timestamp replaced.
|
||||
*
|
||||
* @param timestamp The timestamp of the result record.
|
||||
* @return A new Record instance with all the same attributes (except that the timestamp is replaced).
|
||||
*/
|
||||
public FixedKeyRecord<K, V> withTimestamp(final long timestamp) {
|
||||
return new FixedKeyRecord<>(key, value, timestamp, headers);
|
||||
}
|
||||
|
||||
/**
|
||||
* A convenient way to produce a new record if you only need to change the headers.
|
||||
*
|
||||
* Copies the attributes of this record with the headers replaced.
|
||||
* Also makes a copy of the provided headers.
|
||||
*
|
||||
* See {@link FixedKeyProcessorContext#forward(FixedKeyRecord)} for
|
||||
* considerations around mutability of keys, values, and headers.
|
||||
*
|
||||
* @param headers The headers of the result record.
|
||||
* @return A new Record instance with all the same attributes (except that the headers are replaced).
|
||||
*/
|
||||
public FixedKeyRecord<K, V> withHeaders(final Headers headers) {
|
||||
return new FixedKeyRecord<>(key, value, timestamp, headers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "FixedKeyRecord{" +
|
||||
"key=" + key +
|
||||
", value=" + value +
|
||||
", timestamp=" + timestamp +
|
||||
", headers=" + headers +
|
||||
'}';
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(final Object o) {
|
||||
if (this == o) return true;
|
||||
if (o == null || getClass() != o.getClass()) return false;
|
||||
final FixedKeyRecord<?, ?> record = (FixedKeyRecord<?, ?>) o;
|
||||
return timestamp == record.timestamp &&
|
||||
Objects.equals(key, record.key) &&
|
||||
Objects.equals(value, record.value) &&
|
||||
Objects.equals(headers, record.headers);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(key, value, timestamp, headers);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.streams.processor.api;
|
||||
|
||||
public final class InternalFixedKeyRecordFactory {
|
||||
|
||||
private InternalFixedKeyRecordFactory() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Only allowed way to create {@link FixedKeyRecord}s.
|
||||
* <p/>
|
||||
* DO NOT USE THIS FACTORY OUTSIDE THE FRAMEWORK.
|
||||
* This could produce undesired results by not partitioning record properly.
|
||||
*
|
||||
* @see FixedKeyProcessor
|
||||
*/
|
||||
public static <KIn, VIn> FixedKeyRecord<KIn, VIn> create(final Record<KIn, VIn> record) {
|
||||
return new FixedKeyRecord<>(
|
||||
record.key(),
|
||||
record.value(),
|
||||
record.timestamp(),
|
||||
record.headers()
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,210 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.streams.processor.api;
|
||||
|
||||
import org.apache.kafka.common.serialization.Serde;
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
import org.apache.kafka.streams.StreamsMetrics;
|
||||
import org.apache.kafka.streams.Topology;
|
||||
import org.apache.kafka.streams.processor.Cancellable;
|
||||
import org.apache.kafka.streams.processor.PunctuationType;
|
||||
import org.apache.kafka.streams.processor.Punctuator;
|
||||
import org.apache.kafka.streams.processor.StateStore;
|
||||
import org.apache.kafka.streams.processor.TaskId;
|
||||
import org.apache.kafka.streams.processor.TimestampExtractor;
|
||||
|
||||
import java.io.File;
|
||||
import java.time.Duration;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Processor context interface.
|
||||
*/
|
||||
public interface ProcessingContext {
|
||||
|
||||
/**
|
||||
* Return the application id.
|
||||
*
|
||||
* @return the application id
|
||||
*/
|
||||
String applicationId();
|
||||
|
||||
/**
|
||||
* Return the task id.
|
||||
*
|
||||
* @return the task id
|
||||
*/
|
||||
TaskId taskId();
|
||||
|
||||
/**
|
||||
* Return the metadata of the current record if available. Processors may be invoked to
|
||||
* process a source record from an input topic, to run a scheduled punctuation
|
||||
* (see {@link ProcessingContext#schedule(Duration, PunctuationType, Punctuator)}),
|
||||
* or because a parent processor called {@code forward(Record)}.
|
||||
* <p>
|
||||
* In the case of a punctuation, there is no source record, so this metadata would be
|
||||
* undefined. Note that when a punctuator invokes {@code forward(Record)},
|
||||
* downstream processors will receive the forwarded record as a regular
|
||||
* {@link Processor#process(Record)} or {@link FixedKeyProcessor#process(FixedKeyRecord)} invocation.
|
||||
* In other words, it wouldn't be apparent to
|
||||
* downstream processors whether the record being processed came from an input topic
|
||||
* or punctuation and therefore whether this metadata is defined. This is why
|
||||
* the return type of this method is {@link Optional}.
|
||||
* <p>
|
||||
* If there is any possibility of punctuators upstream, any access
|
||||
* to this field should consider the case of
|
||||
* "<code>recordMetadata().isPresent() == false</code>".
|
||||
* Of course, it would be safest to always guard this condition.
|
||||
*/
|
||||
Optional<RecordMetadata> recordMetadata();
|
||||
|
||||
/**
|
||||
* Return the default key serde.
|
||||
*
|
||||
* @return the key serializer
|
||||
*/
|
||||
Serde<?> keySerde();
|
||||
|
||||
/**
|
||||
* Return the default value serde.
|
||||
*
|
||||
* @return the value serializer
|
||||
*/
|
||||
Serde<?> valueSerde();
|
||||
|
||||
/**
|
||||
* Return the state directory for the partition.
|
||||
*
|
||||
* @return the state directory
|
||||
*/
|
||||
File stateDir();
|
||||
|
||||
/**
|
||||
* Return Metrics instance.
|
||||
*
|
||||
* @return StreamsMetrics
|
||||
*/
|
||||
StreamsMetrics metrics();
|
||||
|
||||
/**
|
||||
* Get the state store given the store name.
|
||||
*
|
||||
* @param name The store name
|
||||
* @param <S> The type or interface of the store to return
|
||||
* @return The state store instance
|
||||
*
|
||||
* @throws ClassCastException if the return type isn't a type or interface of the actual returned store.
|
||||
*/
|
||||
<S extends StateStore> S getStateStore(final String name);
|
||||
|
||||
/**
|
||||
* Schedule a periodic operation for processors. A processor may call this method during
|
||||
* {@link Processor#init(ProcessorContext) initialization},
|
||||
* {@link Processor#process(Record) processing},
|
||||
* {@link FixedKeyProcessor#init(FixedKeyProcessorContext) initialization}, or
|
||||
* {@link FixedKeyProcessor#process(FixedKeyRecord) processing} to
|
||||
* schedule a periodic callback — called a punctuation — to {@link Punctuator#punctuate(long)}.
|
||||
* The type parameter controls what notion of time is used for punctuation:
|
||||
* <ul>
|
||||
* <li>{@link PunctuationType#STREAM_TIME} — uses "stream time", which is advanced by the processing of messages
|
||||
* in accordance with the timestamp as extracted by the {@link TimestampExtractor} in use.
|
||||
* The first punctuation will be triggered by the first record that is processed.
|
||||
* <b>NOTE:</b> Only advanced if messages arrive</li>
|
||||
* <li>{@link PunctuationType#WALL_CLOCK_TIME} — uses system time (the wall-clock time),
|
||||
* which is advanced independent of whether new messages arrive.
|
||||
* The first punctuation will be triggered after interval has elapsed.
|
||||
* <b>NOTE:</b> This is best effort only as its granularity is limited by how long an iteration of the
|
||||
* processing loop takes to complete</li>
|
||||
* </ul>
|
||||
*
|
||||
* <b>Skipping punctuations:</b> Punctuations will not be triggered more than once at any given timestamp.
|
||||
* This means that "missed" punctuation will be skipped.
|
||||
* It's possible to "miss" a punctuation if:
|
||||
* <ul>
|
||||
* <li>with {@link PunctuationType#STREAM_TIME}, when stream time advances more than interval</li>
|
||||
* <li>with {@link PunctuationType#WALL_CLOCK_TIME}, on GC pause, too short interval, ...</li>
|
||||
* </ul>
|
||||
*
|
||||
* @param interval the time interval between punctuations (supported minimum is 1 millisecond)
|
||||
* @param type one of: {@link PunctuationType#STREAM_TIME}, {@link PunctuationType#WALL_CLOCK_TIME}
|
||||
* @param callback a function consuming timestamps representing the current stream or system time
|
||||
* @return a handle allowing cancellation of the punctuation schedule established by this method
|
||||
* @throws IllegalArgumentException if the interval is not representable in milliseconds
|
||||
*/
|
||||
Cancellable schedule(final Duration interval,
|
||||
final PunctuationType type,
|
||||
final Punctuator callback);
|
||||
|
||||
/**
|
||||
* Request a commit.
|
||||
*/
|
||||
void commit();
|
||||
|
||||
/**
|
||||
* Returns all the application config properties as key/value pairs.
|
||||
*
|
||||
* <p> The config properties are defined in the {@link org.apache.kafka.streams.StreamsConfig}
|
||||
* object and associated to the ProcessorContext.
|
||||
*
|
||||
* <p> The type of the values is dependent on the {@link org.apache.kafka.common.config.ConfigDef.Type type} of the property
|
||||
* (e.g. the value of {@link org.apache.kafka.streams.StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG DEFAULT_KEY_SERDE_CLASS_CONFIG}
|
||||
* will be of type {@link Class}, even if it was specified as a String to
|
||||
* {@link org.apache.kafka.streams.StreamsConfig#StreamsConfig(Map) StreamsConfig(Map)}).
|
||||
*
|
||||
* @return all the key/values from the StreamsConfig properties
|
||||
*/
|
||||
Map<String, Object> appConfigs();
|
||||
|
||||
/**
|
||||
* Return all the application config properties with the given key prefix, as key/value pairs
|
||||
* stripping the prefix.
|
||||
*
|
||||
* <p> The config properties are defined in the {@link org.apache.kafka.streams.StreamsConfig}
|
||||
* object and associated to the ProcessorContext.
|
||||
*
|
||||
* @param prefix the properties prefix
|
||||
* @return the key/values matching the given prefix from the StreamsConfig properties.
|
||||
*/
|
||||
Map<String, Object> appConfigsWithPrefix(final String prefix);
|
||||
|
||||
/**
|
||||
* Return the current system timestamp (also called wall-clock time) in milliseconds.
|
||||
*
|
||||
* <p> Note: this method returns the internally cached system timestamp from the Kafka Stream runtime.
|
||||
* Thus, it may return a different value compared to {@code System.currentTimeMillis()}.
|
||||
*
|
||||
* @return the current system timestamp in milliseconds
|
||||
*/
|
||||
long currentSystemTimeMs();
|
||||
|
||||
/**
|
||||
* Return the current stream-time in milliseconds.
|
||||
*
|
||||
* <p> Stream-time is the maximum observed {@link TimestampExtractor record timestamp} so far
|
||||
* (including the currently processed record), i.e., it can be considered a high-watermark.
|
||||
* Stream-time is tracked on a per-task basis and is preserved across restarts and during task migration.
|
||||
*
|
||||
* <p> Note: this method is not supported for global processors (cf. {@link Topology#addGlobalStore} (...)
|
||||
* and {@link StreamsBuilder#addGlobalStore} (...),
|
||||
* because there is no concept of stream-time for this case.
|
||||
* Calling this method in a global processor will result in an {@link UnsupportedOperationException}.
|
||||
*
|
||||
* @return the current stream-time in milliseconds
|
||||
*/
|
||||
long currentStreamTimeMs();
|
||||
}
|
|
@ -16,140 +16,13 @@
|
|||
*/
|
||||
package org.apache.kafka.streams.processor.api;
|
||||
|
||||
import org.apache.kafka.common.serialization.Serde;
|
||||
import org.apache.kafka.streams.StreamsBuilder;
|
||||
import org.apache.kafka.streams.StreamsMetrics;
|
||||
import org.apache.kafka.streams.Topology;
|
||||
import org.apache.kafka.streams.processor.Cancellable;
|
||||
import org.apache.kafka.streams.processor.PunctuationType;
|
||||
import org.apache.kafka.streams.processor.Punctuator;
|
||||
import org.apache.kafka.streams.processor.StateStore;
|
||||
import org.apache.kafka.streams.processor.TaskId;
|
||||
import org.apache.kafka.streams.processor.TimestampExtractor;
|
||||
|
||||
import java.io.File;
|
||||
import java.time.Duration;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
|
||||
/**
|
||||
* Processor context interface.
|
||||
* Processor context interface for {@link Record}.
|
||||
*
|
||||
* @param <KForward> a bound on the types of keys that may be forwarded
|
||||
* @param <VForward> a bound on the types of values that may be forwarded
|
||||
*/
|
||||
public interface ProcessorContext<KForward, VForward> {
|
||||
|
||||
/**
|
||||
* Return the application id.
|
||||
*
|
||||
* @return the application id
|
||||
*/
|
||||
String applicationId();
|
||||
|
||||
/**
|
||||
* Return the task id.
|
||||
*
|
||||
* @return the task id
|
||||
*/
|
||||
TaskId taskId();
|
||||
|
||||
/**
|
||||
* Return the metadata of the current record if available. Processors may be invoked to
|
||||
* process a source record from an input topic, to run a scheduled punctuation
|
||||
* (see {@link ProcessorContext#schedule(Duration, PunctuationType, Punctuator)}),
|
||||
* or because a parent processor called {@link ProcessorContext#forward(Record)}.
|
||||
* <p>
|
||||
* In the case of a punctuation, there is no source record, so this metadata would be
|
||||
* undefined. Note that when a punctuator invokes {@link ProcessorContext#forward(Record)},
|
||||
* downstream processors will receive the forwarded record as a regular
|
||||
* {@link Processor#process(Record)} invocation. In other words, it wouldn't be apparent to
|
||||
* downstream processors whether or not the record being processed came from an input topic
|
||||
* or punctuation and therefore whether or not this metadata is defined. This is why
|
||||
* the return type of this method is {@link Optional}.
|
||||
* <p>
|
||||
* If there is any possibility of punctuators upstream, any access
|
||||
* to this field should consider the case of
|
||||
* "<code>recordMetadata().isPresent() == false</code>".
|
||||
* Of course, it would be safest to always guard this condition.
|
||||
*/
|
||||
Optional<RecordMetadata> recordMetadata();
|
||||
|
||||
/**
|
||||
* Return the default key serde.
|
||||
*
|
||||
* @return the key serializer
|
||||
*/
|
||||
Serde<?> keySerde();
|
||||
|
||||
/**
|
||||
* Return the default value serde.
|
||||
*
|
||||
* @return the value serializer
|
||||
*/
|
||||
Serde<?> valueSerde();
|
||||
|
||||
/**
|
||||
* Return the state directory for the partition.
|
||||
*
|
||||
* @return the state directory
|
||||
*/
|
||||
File stateDir();
|
||||
|
||||
/**
|
||||
* Return Metrics instance.
|
||||
*
|
||||
* @return StreamsMetrics
|
||||
*/
|
||||
StreamsMetrics metrics();
|
||||
|
||||
/**
|
||||
* Get the state store given the store name.
|
||||
*
|
||||
* @param name The store name
|
||||
* @param <S> The type or interface of the store to return
|
||||
* @return The state store instance
|
||||
*
|
||||
* @throws ClassCastException if the return type isn't a type or interface of the actual returned store.
|
||||
*/
|
||||
<S extends StateStore> S getStateStore(final String name);
|
||||
|
||||
/**
|
||||
* Schedule a periodic operation for processors. A processor may call this method during
|
||||
* {@link Processor#init(ProcessorContext) initialization} or
|
||||
* {@link Processor#process(Record)} processing} to
|
||||
* schedule a periodic callback — called a punctuation — to {@link Punctuator#punctuate(long)}.
|
||||
* The type parameter controls what notion of time is used for punctuation:
|
||||
* <ul>
|
||||
* <li>{@link PunctuationType#STREAM_TIME} — uses "stream time", which is advanced by the processing of messages
|
||||
* in accordance with the timestamp as extracted by the {@link TimestampExtractor} in use.
|
||||
* The first punctuation will be triggered by the first record that is processed.
|
||||
* <b>NOTE:</b> Only advanced if messages arrive</li>
|
||||
* <li>{@link PunctuationType#WALL_CLOCK_TIME} — uses system time (the wall-clock time),
|
||||
* which is advanced independent of whether new messages arrive.
|
||||
* The first punctuation will be triggered after interval has elapsed.
|
||||
* <b>NOTE:</b> This is best effort only as its granularity is limited by how long an iteration of the
|
||||
* processing loop takes to complete</li>
|
||||
* </ul>
|
||||
*
|
||||
* <b>Skipping punctuations:</b> Punctuations will not be triggered more than once at any given timestamp.
|
||||
* This means that "missed" punctuation will be skipped.
|
||||
* It's possible to "miss" a punctuation if:
|
||||
* <ul>
|
||||
* <li>with {@link PunctuationType#STREAM_TIME}, when stream time advances more than interval</li>
|
||||
* <li>with {@link PunctuationType#WALL_CLOCK_TIME}, on GC pause, too short interval, ...</li>
|
||||
* </ul>
|
||||
*
|
||||
* @param interval the time interval between punctuations (supported minimum is 1 millisecond)
|
||||
* @param type one of: {@link PunctuationType#STREAM_TIME}, {@link PunctuationType#WALL_CLOCK_TIME}
|
||||
* @param callback a function consuming timestamps representing the current stream or system time
|
||||
* @return a handle allowing cancellation of the punctuation schedule established by this method
|
||||
* @throws IllegalArgumentException if the interval is not representable in milliseconds
|
||||
*/
|
||||
Cancellable schedule(final Duration interval,
|
||||
final PunctuationType type,
|
||||
final Punctuator callback);
|
||||
|
||||
public interface ProcessorContext<KForward, VForward> extends ProcessingContext {
|
||||
/**
|
||||
* Forward a record to all child processors.
|
||||
* <p>
|
||||
|
@ -217,62 +90,4 @@ public interface ProcessorContext<KForward, VForward> {
|
|||
* @see ProcessorContext#forward(Record)
|
||||
*/
|
||||
<K extends KForward, V extends VForward> void forward(Record<K, V> record, final String childName);
|
||||
|
||||
/**
|
||||
* Request a commit.
|
||||
*/
|
||||
void commit();
|
||||
|
||||
/**
|
||||
* Returns all the application config properties as key/value pairs.
|
||||
*
|
||||
* <p> The config properties are defined in the {@link org.apache.kafka.streams.StreamsConfig}
|
||||
* object and associated to the ProcessorContext.
|
||||
*
|
||||
* <p> The type of the values is dependent on the {@link org.apache.kafka.common.config.ConfigDef.Type type} of the property
|
||||
* (e.g. the value of {@link org.apache.kafka.streams.StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG DEFAULT_KEY_SERDE_CLASS_CONFIG}
|
||||
* will be of type {@link Class}, even if it was specified as a String to
|
||||
* {@link org.apache.kafka.streams.StreamsConfig#StreamsConfig(Map) StreamsConfig(Map)}).
|
||||
*
|
||||
* @return all the key/values from the StreamsConfig properties
|
||||
*/
|
||||
Map<String, Object> appConfigs();
|
||||
|
||||
/**
|
||||
* Return all the application config properties with the given key prefix, as key/value pairs
|
||||
* stripping the prefix.
|
||||
*
|
||||
* <p> The config properties are defined in the {@link org.apache.kafka.streams.StreamsConfig}
|
||||
* object and associated to the ProcessorContext.
|
||||
*
|
||||
* @param prefix the properties prefix
|
||||
* @return the key/values matching the given prefix from the StreamsConfig properties.
|
||||
*/
|
||||
Map<String, Object> appConfigsWithPrefix(final String prefix);
|
||||
|
||||
/**
|
||||
* Return the current system timestamp (also called wall-clock time) in milliseconds.
|
||||
*
|
||||
* <p> Note: this method returns the internally cached system timestamp from the Kafka Stream runtime.
|
||||
* Thus, it may return a different value compared to {@code System.currentTimeMillis()}.
|
||||
*
|
||||
* @return the current system timestamp in milliseconds
|
||||
*/
|
||||
long currentSystemTimeMs();
|
||||
|
||||
/**
|
||||
* Return the current stream-time in milliseconds.
|
||||
*
|
||||
* <p> Stream-time is the maximum observed {@link TimestampExtractor record timestamp} so far
|
||||
* (including the currently processed record), i.e., it can be considered a high-watermark.
|
||||
* Stream-time is tracked on a per-task basis and is preserved across restarts and during task migration.
|
||||
*
|
||||
* <p> Note: this method is not supported for global processors (cf. {@link Topology#addGlobalStore} (...)
|
||||
* and {@link StreamsBuilder#addGlobalStore} (...),
|
||||
* because there is no concept of stream-time for this case.
|
||||
* Calling this method in a global processor will result in an {@link UnsupportedOperationException}.
|
||||
*
|
||||
* @return the current stream-time in milliseconds
|
||||
*/
|
||||
long currentStreamTimeMs();
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.kafka.streams.processor.Punctuator;
|
|||
import org.apache.kafka.streams.processor.StateStore;
|
||||
import org.apache.kafka.streams.processor.TaskId;
|
||||
import org.apache.kafka.streams.processor.To;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyRecord;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
|
||||
import org.apache.kafka.streams.query.Position;
|
||||
|
@ -96,6 +97,19 @@ public class GlobalProcessorContextImpl extends AbstractProcessorContext<Object,
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K, V> void forward(final FixedKeyRecord<K, V> record) {
|
||||
forward(new Record<>(record.key(), record.value(), record.timestamp(), record.headers()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K, V> void forward(final FixedKeyRecord<K, V> record, final String childName) {
|
||||
forward(
|
||||
new Record<>(record.key(), record.value(), record.timestamp(), record.headers()),
|
||||
childName
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void commit() {
|
||||
//no-op
|
||||
|
|
|
@ -31,12 +31,15 @@ import org.apache.kafka.streams.state.internals.ThreadCache;
|
|||
import org.apache.kafka.streams.state.internals.ThreadCache.DirtyEntryFlushListener;
|
||||
|
||||
/**
|
||||
* For internal use so we can update the {@link RecordContext} and current
|
||||
* For internal use, so we can update the {@link RecordContext} and current
|
||||
* {@link ProcessorNode} when we are forwarding items that have been evicted or flushed from
|
||||
* {@link ThreadCache}
|
||||
*/
|
||||
public interface InternalProcessorContext<KOut, VOut>
|
||||
extends ProcessorContext, org.apache.kafka.streams.processor.api.ProcessorContext<KOut, VOut>, StateStoreContext {
|
||||
extends ProcessorContext,
|
||||
org.apache.kafka.streams.processor.api.ProcessorContext<KOut, VOut>,
|
||||
org.apache.kafka.streams.processor.api.FixedKeyProcessorContext<KOut, VOut>,
|
||||
StateStoreContext {
|
||||
|
||||
BytesSerializer BYTES_KEY_SERIALIZER = new BytesSerializer();
|
||||
ByteArraySerializer BYTEARRAY_VALUE_SERIALIZER = new ByteArraySerializer();
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.kafka.streams.processor.StateStore;
|
|||
import org.apache.kafka.streams.processor.StreamPartitioner;
|
||||
import org.apache.kafka.streams.processor.TimestampExtractor;
|
||||
import org.apache.kafka.streams.processor.TopicNameExtractor;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology;
|
||||
import org.apache.kafka.streams.processor.internals.namedtopology.NamedTopology;
|
||||
|
@ -247,6 +248,32 @@ public class InternalTopologyBuilder {
|
|||
}
|
||||
}
|
||||
|
||||
private static class FixedKeyProcessorNodeFactory<KIn, VIn, VOut> extends ProcessorNodeFactory<KIn, VIn, KIn, VOut> {
|
||||
private final FixedKeyProcessorSupplier<KIn, VIn, VOut> supplier;
|
||||
private final Set<String> stateStoreNames = new HashSet<>();
|
||||
|
||||
FixedKeyProcessorNodeFactory(final String name,
|
||||
final String[] predecessors,
|
||||
final FixedKeyProcessorSupplier<KIn, VIn, VOut> supplier) {
|
||||
super(name, predecessors.clone(), null);
|
||||
this.supplier = supplier;
|
||||
}
|
||||
|
||||
public void addStateStore(final String stateStoreName) {
|
||||
stateStoreNames.add(stateStoreName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public ProcessorNode<KIn, VIn, KIn, VOut> build() {
|
||||
return new ProcessorNode<>(name, supplier.get(), stateStoreNames);
|
||||
}
|
||||
|
||||
@Override
|
||||
Processor describe() {
|
||||
return new Processor(name, new HashSet<>(stateStoreNames));
|
||||
}
|
||||
}
|
||||
|
||||
// Map from topics to their matched regex patterns, this is to ensure one topic is passed through on source node
|
||||
// even if it can be matched by multiple regex patterns. Only used by SourceNodeFactory
|
||||
private final Map<String, Pattern> topicToPatterns = new HashMap<>();
|
||||
|
@ -544,6 +571,36 @@ public class InternalTopologyBuilder {
|
|||
nodeGroups = null;
|
||||
}
|
||||
|
||||
public final <KIn, VIn, VOut> void addProcessor(final String name,
|
||||
final FixedKeyProcessorSupplier<KIn, VIn, VOut> supplier,
|
||||
final String... predecessorNames) {
|
||||
Objects.requireNonNull(name, "name must not be null");
|
||||
Objects.requireNonNull(supplier, "supplier must not be null");
|
||||
Objects.requireNonNull(predecessorNames, "predecessor names must not be null");
|
||||
ApiUtils.checkSupplier(supplier);
|
||||
if (nodeFactories.containsKey(name)) {
|
||||
throw new TopologyException("Processor " + name + " is already added.");
|
||||
}
|
||||
if (predecessorNames.length == 0) {
|
||||
throw new TopologyException("Processor " + name + " must have at least one parent");
|
||||
}
|
||||
|
||||
for (final String predecessor : predecessorNames) {
|
||||
Objects.requireNonNull(predecessor, "predecessor name must not be null");
|
||||
if (predecessor.equals(name)) {
|
||||
throw new TopologyException("Processor " + name + " cannot be a predecessor of itself.");
|
||||
}
|
||||
if (!nodeFactories.containsKey(predecessor)) {
|
||||
throw new TopologyException("Predecessor processor " + predecessor + " is not added yet for " + name);
|
||||
}
|
||||
}
|
||||
|
||||
nodeFactories.put(name, new FixedKeyProcessorNodeFactory<>(name, predecessorNames, supplier));
|
||||
nodeGrouper.add(name);
|
||||
nodeGrouper.unite(name, predecessorNames);
|
||||
nodeGroups = null;
|
||||
}
|
||||
|
||||
public final void addStateStore(final StoreBuilder<?> storeBuilder,
|
||||
final String... processorNames) {
|
||||
addStateStore(storeBuilder, false, processorNames);
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.kafka.streams.processor.Punctuator;
|
|||
import org.apache.kafka.streams.processor.StateStore;
|
||||
import org.apache.kafka.streams.processor.TaskId;
|
||||
import org.apache.kafka.streams.processor.To;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyRecord;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
import org.apache.kafka.streams.processor.internals.Task.TaskType;
|
||||
import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
|
||||
|
@ -208,6 +209,19 @@ public class ProcessorContextImpl extends AbstractProcessorContext<Object, Objec
|
|||
forward(toForward, toInternal.child());
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K, V> void forward(final FixedKeyRecord<K, V> record) {
|
||||
forward(new Record<>(record.key(), record.value(), record.timestamp(), record.headers()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K, V> void forward(final FixedKeyRecord<K, V> record, final String childName) {
|
||||
forward(
|
||||
new Record<>(record.key(), record.value(), record.timestamp(), record.headers()),
|
||||
childName
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K, V> void forward(final Record<K, V> record) {
|
||||
forward(record, null);
|
||||
|
|
|
@ -20,6 +20,9 @@ import org.apache.kafka.common.utils.SystemTime;
|
|||
import org.apache.kafka.common.utils.Time;
|
||||
import org.apache.kafka.streams.errors.StreamsException;
|
||||
import org.apache.kafka.streams.processor.Punctuator;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessor;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessorContext;
|
||||
import org.apache.kafka.streams.processor.api.InternalFixedKeyRecordFactory;
|
||||
import org.apache.kafka.streams.processor.api.Processor;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
|
||||
|
@ -35,12 +38,13 @@ public class ProcessorNode<KIn, VIn, KOut, VOut> {
|
|||
private final Map<String, ProcessorNode<KOut, VOut, ?, ?>> childByName;
|
||||
|
||||
private final Processor<KIn, VIn, KOut, VOut> processor;
|
||||
private final FixedKeyProcessor<KIn, VIn, VOut> fixedKeyProcessor;
|
||||
private final String name;
|
||||
private final Time time;
|
||||
|
||||
public final Set<String> stateStores;
|
||||
|
||||
private InternalProcessorContext internalProcessorContext;
|
||||
private InternalProcessorContext<KOut, VOut> internalProcessorContext;
|
||||
private String threadId;
|
||||
|
||||
private boolean closed = true;
|
||||
|
@ -55,6 +59,7 @@ public class ProcessorNode<KIn, VIn, KOut, VOut> {
|
|||
|
||||
this.name = name;
|
||||
this.processor = processor;
|
||||
this.fixedKeyProcessor = null;
|
||||
this.children = new ArrayList<>();
|
||||
this.childByName = new HashMap<>();
|
||||
this.stateStores = stateStores;
|
||||
|
@ -62,11 +67,12 @@ public class ProcessorNode<KIn, VIn, KOut, VOut> {
|
|||
}
|
||||
|
||||
public ProcessorNode(final String name,
|
||||
@SuppressWarnings("deprecation") final org.apache.kafka.streams.processor.Processor<KIn, VIn> processor,
|
||||
final FixedKeyProcessor<KIn, VIn, VOut> processor,
|
||||
final Set<String> stateStores) {
|
||||
|
||||
this.name = name;
|
||||
this.processor = ProcessorAdapter.adapt(processor);
|
||||
this.processor = null;
|
||||
this.fixedKeyProcessor = processor;
|
||||
this.children = new ArrayList<>();
|
||||
this.childByName = new HashMap<>();
|
||||
this.stateStores = stateStores;
|
||||
|
@ -77,10 +83,6 @@ public class ProcessorNode<KIn, VIn, KOut, VOut> {
|
|||
return name;
|
||||
}
|
||||
|
||||
public final Processor<KIn, VIn, KOut, VOut> processor() {
|
||||
return processor;
|
||||
}
|
||||
|
||||
public List<ProcessorNode<KOut, VOut, ?, ?>> children() {
|
||||
return children;
|
||||
}
|
||||
|
@ -104,6 +106,11 @@ public class ProcessorNode<KIn, VIn, KOut, VOut> {
|
|||
if (processor != null) {
|
||||
processor.init(context);
|
||||
}
|
||||
if (fixedKeyProcessor != null) {
|
||||
@SuppressWarnings("unchecked") final FixedKeyProcessorContext<KIn, VOut> fixedKeyProcessorContext =
|
||||
(FixedKeyProcessorContext<KIn, VOut>) context;
|
||||
fixedKeyProcessor.init(fixedKeyProcessorContext);
|
||||
}
|
||||
} catch (final Exception e) {
|
||||
throw new StreamsException(String.format("failed to initialize processor %s", name), e);
|
||||
}
|
||||
|
@ -120,6 +127,9 @@ public class ProcessorNode<KIn, VIn, KOut, VOut> {
|
|||
if (processor != null) {
|
||||
processor.close();
|
||||
}
|
||||
if (fixedKeyProcessor != null) {
|
||||
fixedKeyProcessor.close();
|
||||
}
|
||||
internalProcessorContext.metrics().removeAllNodeLevelSensors(
|
||||
threadId,
|
||||
internalProcessorContext.taskId().toString(),
|
||||
|
@ -143,7 +153,17 @@ public class ProcessorNode<KIn, VIn, KOut, VOut> {
|
|||
throwIfClosed();
|
||||
|
||||
try {
|
||||
processor.process(record);
|
||||
if (processor != null) {
|
||||
processor.process(record);
|
||||
} else if (fixedKeyProcessor != null) {
|
||||
fixedKeyProcessor.process(
|
||||
InternalFixedKeyRecordFactory.create(record)
|
||||
);
|
||||
} else {
|
||||
throw new IllegalStateException(
|
||||
"neither the processor nor the fixed key processor were set."
|
||||
);
|
||||
}
|
||||
} catch (final ClassCastException e) {
|
||||
final String keyClass = record.key() == null ? "unknown because key is null" : record.key().getClass().getName();
|
||||
final String valueClass = record.value() == null ? "unknown because value is null" : record.value().getClass().getName();
|
||||
|
|
|
@ -622,6 +622,7 @@ public class StreamsBuilderTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldUseSpecifiedNameForTransform() {
|
||||
builder.stream(STREAM_TOPIC).transform(() -> null, Named.as(STREAM_OPERATION_NAME));
|
||||
builder.build();
|
||||
|
@ -630,6 +631,7 @@ public class StreamsBuilderTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldUseSpecifiedNameForTransformValues() {
|
||||
builder.stream(STREAM_TOPIC).transformValues(() -> new NoopValueTransformer<>(), Named.as(STREAM_OPERATION_NAME));
|
||||
builder.build();
|
||||
|
@ -638,6 +640,7 @@ public class StreamsBuilderTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldUseSpecifiedNameForTransformValuesWithKey() {
|
||||
builder.stream(STREAM_TOPIC).transformValues(() -> new NoopValueTransformerWithKey<>(), Named.as(STREAM_OPERATION_NAME));
|
||||
builder.build();
|
||||
|
@ -944,6 +947,7 @@ public class StreamsBuilderTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldUseSpecifiedNameForFlatTransformValueOperation() {
|
||||
builder.stream(STREAM_TOPIC).flatTransformValues(() -> new NoopValueTransformer<>(), Named.as(STREAM_OPERATION_NAME));
|
||||
builder.build();
|
||||
|
@ -952,7 +956,7 @@ public class StreamsBuilderTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings({"unchecked", "rawtypes"})
|
||||
@SuppressWarnings({"unchecked", "rawtypes", "deprecation"})
|
||||
public void shouldUseSpecifiedNameForFlatTransformValueWithKeyOperation() {
|
||||
builder.stream(STREAM_TOPIC).flatTransformValues(() -> new NoopValueTransformerWithKey(), Named.as(STREAM_OPERATION_NAME));
|
||||
builder.build();
|
||||
|
|
|
@ -421,6 +421,7 @@ public class AdjustStreamThreadCountTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldResizeCacheAndInputBufferAfterThreadReplacement() throws InterruptedException {
|
||||
final long totalCacheBytes = 10L;
|
||||
final long maxBufferBytes = 100L;
|
||||
|
|
|
@ -862,6 +862,7 @@ public class EosV2UpgradeIntegrationTest {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
private KafkaStreams getKafkaStreams(final String appDir,
|
||||
final String processingGuarantee) {
|
||||
final StreamsBuilder builder = new StreamsBuilder();
|
||||
|
|
|
@ -115,6 +115,7 @@ public class KStreamTransformIntegrationTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldTransform() {
|
||||
builder.addStateStore(storeBuilder());
|
||||
|
||||
|
@ -133,6 +134,7 @@ public class KStreamTransformIntegrationTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldTransformWithConnectedStoreProvider() {
|
||||
stream
|
||||
.transform(new TransformerSupplier<Integer, Integer, KeyValue<Integer, Integer>>() {
|
||||
|
@ -185,6 +187,7 @@ public class KStreamTransformIntegrationTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldFlatTransform() {
|
||||
builder.addStateStore(storeBuilder());
|
||||
|
||||
|
@ -215,6 +218,7 @@ public class KStreamTransformIntegrationTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldFlatTransformWithConnectedStoreProvider() {
|
||||
stream
|
||||
.flatTransform(new TransformerSupplier<Integer, Integer, Iterable<KeyValue<Integer, Integer>>>() {
|
||||
|
@ -275,6 +279,7 @@ public class KStreamTransformIntegrationTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldTransformValuesWithValueTransformerWithKey() {
|
||||
builder.addStateStore(storeBuilder());
|
||||
|
||||
|
@ -293,6 +298,7 @@ public class KStreamTransformIntegrationTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldTransformValuesWithValueTransformerWithKeyWithConnectedStoreProvider() {
|
||||
stream
|
||||
.transformValues(new ValueTransformerWithKeySupplier<Integer, Integer, Integer>() {
|
||||
|
@ -331,6 +337,7 @@ public class KStreamTransformIntegrationTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldTransformValuesWithValueTransformerWithoutKey() {
|
||||
builder.addStateStore(storeBuilder());
|
||||
|
||||
|
@ -349,6 +356,7 @@ public class KStreamTransformIntegrationTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldTransformValuesWithValueTransformerWithoutKeyWithConnectedStoreProvider() {
|
||||
stream
|
||||
.transformValues(new ValueTransformerSupplier<Integer, Integer>() {
|
||||
|
@ -400,6 +408,7 @@ public class KStreamTransformIntegrationTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldFlatTransformValuesWithKey() {
|
||||
builder.addStateStore(storeBuilder());
|
||||
|
||||
|
@ -430,6 +439,7 @@ public class KStreamTransformIntegrationTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldFlatTransformValuesWithKeyWithConnectedStoreProvider() {
|
||||
stream
|
||||
.flatTransformValues(new ValueTransformerWithKeySupplier<Integer, Integer, Iterable<Integer>>() {
|
||||
|
@ -493,6 +503,7 @@ public class KStreamTransformIntegrationTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldFlatTransformValuesWithValueTransformerWithoutKey() {
|
||||
builder.addStateStore(storeBuilder());
|
||||
|
||||
|
@ -523,6 +534,7 @@ public class KStreamTransformIntegrationTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldFlatTransformValuesWithValueTransformerWithoutKeyWithConnectedStoreProvider() {
|
||||
stream
|
||||
.flatTransformValues(new ValueTransformerSupplier<Integer, Iterable<Integer>>() {
|
||||
|
|
|
@ -95,6 +95,7 @@ public class StandbyTaskCreationIntegrationTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotCreateAnyStandByTasksForStateStoreWithLoggingDisabled() throws Exception {
|
||||
final StreamsBuilder builder = new StreamsBuilder();
|
||||
final String stateStoreName = "myTransformState";
|
||||
|
|
|
@ -342,6 +342,7 @@ public class StandbyTaskEOSIntegrationTest {
|
|||
);
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
private KafkaStreams buildWithDeduplicationTopology(final String stateDirPath) {
|
||||
final StreamsBuilder builder = new StreamsBuilder();
|
||||
|
||||
|
|
|
@ -125,6 +125,7 @@ public class SuppressionDurabilityIntegrationTest {
|
|||
public String processingGuaranteee;
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldRecoverBufferAfterShutdown() {
|
||||
final String testId = safeUniqueTestName(getClass(), testName);
|
||||
final String appId = "appId_" + testId;
|
||||
|
|
|
@ -16,6 +16,8 @@
|
|||
*/
|
||||
package org.apache.kafka.streams.kstream.internals;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.Set;
|
||||
import org.apache.kafka.common.serialization.Serde;
|
||||
import org.apache.kafka.common.serialization.Serdes;
|
||||
import org.apache.kafka.common.serialization.StringDeserializer;
|
||||
|
@ -56,12 +58,19 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier;
|
|||
import org.apache.kafka.streams.processor.FailOnInvalidTimestamp;
|
||||
import org.apache.kafka.streams.processor.ProcessorContext;
|
||||
import org.apache.kafka.streams.processor.TopicNameExtractor;
|
||||
import org.apache.kafka.streams.processor.api.ContextualFixedKeyProcessor;
|
||||
import org.apache.kafka.streams.processor.api.ContextualProcessor;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyRecord;
|
||||
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
import org.apache.kafka.streams.processor.internals.ProcessorTopology;
|
||||
import org.apache.kafka.streams.processor.internals.SourceNode;
|
||||
import org.apache.kafka.streams.state.KeyValueStore;
|
||||
import org.apache.kafka.streams.state.StoreBuilder;
|
||||
import org.apache.kafka.streams.state.Stores;
|
||||
import org.apache.kafka.streams.test.TestRecord;
|
||||
import org.apache.kafka.test.MockApiFixedKeyProcessorSupplier;
|
||||
import org.apache.kafka.test.MockApiProcessor;
|
||||
import org.apache.kafka.test.MockApiProcessorSupplier;
|
||||
import org.apache.kafka.test.MockMapper;
|
||||
|
@ -105,6 +114,7 @@ public class KStreamImplTest {
|
|||
|
||||
private final Consumed<String, String> stringConsumed = Consumed.with(Serdes.String(), Serdes.String());
|
||||
private final MockApiProcessorSupplier<String, String, Void, Void> processorSupplier = new MockApiProcessorSupplier<>();
|
||||
private final MockApiFixedKeyProcessorSupplier<String, String, Void> fixedKeyProcessorSupplier = new MockApiFixedKeyProcessorSupplier<>();
|
||||
private final TransformerSupplier<String, String, KeyValue<String, String>> transformerSupplier =
|
||||
() -> new Transformer<String, String, KeyValue<String, String>>() {
|
||||
@Override
|
||||
|
@ -1756,6 +1766,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullTransformerSupplierOnTransform() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1764,6 +1775,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullTransformerSupplierOnTransformWithStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1772,6 +1784,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullTransformerSupplierOnTransformWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1780,6 +1793,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullTransformerSupplierOnTransformWithNamedAndStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1788,6 +1802,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNamesOnTransform() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1796,6 +1811,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNameOnTransform() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1804,6 +1820,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNamesOnTransformWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1812,6 +1829,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNameOnTransformWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1820,6 +1838,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullNamedOnTransform() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1828,6 +1847,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullNamedOnTransformWithStoreName() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1836,6 +1856,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowBadTransformerSupplierOnFlatTransform() {
|
||||
final Transformer<String, String, Iterable<KeyValue<String, String>>> transformer = flatTransformerSupplier.get();
|
||||
final IllegalArgumentException exception = assertThrows(
|
||||
|
@ -1846,6 +1867,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowBadTransformerSupplierOnFlatTransformWithStores() {
|
||||
final Transformer<String, String, Iterable<KeyValue<String, String>>> transformer = flatTransformerSupplier.get();
|
||||
final IllegalArgumentException exception = assertThrows(
|
||||
|
@ -1856,6 +1878,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowBadTransformerSupplierOnFlatTransformWithNamed() {
|
||||
final Transformer<String, String, Iterable<KeyValue<String, String>>> transformer = flatTransformerSupplier.get();
|
||||
final IllegalArgumentException exception = assertThrows(
|
||||
|
@ -1866,6 +1889,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowBadTransformerSupplierOnFlatTransformWithNamedAndStores() {
|
||||
final Transformer<String, String, Iterable<KeyValue<String, String>>> transformer = flatTransformerSupplier.get();
|
||||
final IllegalArgumentException exception = assertThrows(
|
||||
|
@ -1876,6 +1900,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullTransformerSupplierOnFlatTransform() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1884,6 +1909,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullTransformerSupplierOnFlatTransformWithStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1892,6 +1918,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullTransformerSupplierOnFlatTransformWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1900,6 +1927,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullTransformerSupplierOnFlatTransformWithNamedAndStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1908,6 +1936,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNamesOnFlatTransform() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1916,6 +1945,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNameOnFlatTransform() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1924,6 +1954,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNamesOnFlatTransformWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1932,6 +1963,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNameOnFlatTransformWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1940,6 +1972,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullNamedOnFlatTransform() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1948,6 +1981,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullNamedOnFlatTransformWithStoreName() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1956,6 +1990,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowBadTransformerSupplierOnTransformValues() {
|
||||
final ValueTransformer<String, String> transformer = valueTransformerSupplier.get();
|
||||
final IllegalArgumentException exception = assertThrows(
|
||||
|
@ -1966,6 +2001,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowBadTransformerSupplierOnTransformValuesWithNamed() {
|
||||
final ValueTransformer<String, String> transformer = valueTransformerSupplier.get();
|
||||
final IllegalArgumentException exception = assertThrows(
|
||||
|
@ -1976,6 +2012,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerSupplierOnTransformValues() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -1984,6 +2021,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowBadValueTransformerWithKeySupplierOnTransformValues() {
|
||||
final ValueTransformerWithKey<String, String, String> transformer = valueTransformerWithKeySupplier.get();
|
||||
final IllegalArgumentException exception = assertThrows(
|
||||
|
@ -1994,6 +2032,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowBadValueTransformerWithKeySupplierOnTransformValuesWithNamed() {
|
||||
final ValueTransformerWithKey<String, String, String> transformer = valueTransformerWithKeySupplier.get();
|
||||
final IllegalArgumentException exception = assertThrows(
|
||||
|
@ -2004,6 +2043,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerWithKeySupplierOnTransformValues() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2012,6 +2052,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerSupplierOnTransformValuesWithStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2022,6 +2063,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerWithKeySupplierOnTransformValuesWithStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2032,6 +2074,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerSupplierOnTransformValuesWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2042,6 +2085,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerWithKeySupplierOnTransformValuesWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2052,6 +2096,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerSupplierOnTransformValuesWithNamedAndStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2063,6 +2108,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerWithKeySupplierOnTransformValuesWithNamedAndStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2074,6 +2120,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNamesOnTransformValuesWithValueTransformerSupplier() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2084,6 +2131,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNamesOnTransformValuesWithValueTransformerWithKeySupplier() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2094,6 +2142,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNameOnTransformValuesWithValueTransformerSupplier() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2103,6 +2152,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNameOnTransformValuesWithValueTransformerWithKeySupplier() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2113,6 +2163,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNamesOnTransformValuesWithValueTransformerSupplierWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2124,6 +2175,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNamesOnTransformValuesWithValueTransformerWithKeySupplierWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2135,6 +2187,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNameOnTransformValuesWithValueTransformerSupplierWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2146,6 +2199,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNameOnTransformValuesWithValueTransformerWithKeySupplierWithName() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2157,6 +2211,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullNamedOnTransformValuesWithValueTransformerSupplier() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2167,6 +2222,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullNamedOnTransformValuesWithValueTransformerWithKeySupplier() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2177,6 +2233,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullNamedOnTransformValuesWithValueTransformerSupplierAndStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2188,6 +2245,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullNamedOnTransformValuesWithValueTransformerWithKeySupplierAndStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2199,6 +2257,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerSupplierOnFlatTransformValues() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2207,6 +2266,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerWithKeySupplierOnFlatTransformValues() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2215,6 +2275,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerSupplierOnFlatTransformValuesWithStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2225,6 +2286,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerWithKeySupplierOnFlatTransformValuesWithStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2235,6 +2297,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerSupplierOnFlatTransformValuesWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2245,6 +2308,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerWithKeySupplierOnFlatTransformValuesWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2255,6 +2319,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerSupplierOnFlatTransformValuesWithNamedAndStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2266,6 +2331,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullValueTransformerWithKeySupplierOnFlatTransformValuesWithNamedAndStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2277,6 +2343,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNamesOnFlatTransformValuesWithFlatValueSupplier() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2287,6 +2354,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNamesOnFlatTransformValuesWithFlatValueWithKeySupplier() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2297,6 +2365,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNameOnFlatTransformValuesWithFlatValueSupplier() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2307,6 +2376,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNameOnFlatTransformValuesWithFlatValueWithKeySupplier() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2317,6 +2387,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNamesOnFlatTransformValuesWithFlatValueSupplierAndNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2328,6 +2399,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNamesOnFlatTransformValuesWithFlatValueWithKeySupplierAndNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2339,6 +2411,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNameOnFlatTransformValuesWithFlatValueSupplierAndNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2350,6 +2423,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullStoreNameOnFlatTransformValuesWithFlatValueWithKeySupplierAndNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2361,6 +2435,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullNamedOnFlatTransformValuesWithFlatValueSupplier() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2371,6 +2446,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullNamedOnFlatTransformValuesWithFlatValueWithKeySupplier() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2381,6 +2457,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullNamedOnFlatTransformValuesWithFlatValueSupplierAndStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2392,6 +2469,7 @@ public class KStreamImplTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotAllowNullNamedOnFlatTransformValuesWithFlatValueWithKeySupplierAndStore() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
|
@ -2485,6 +2563,88 @@ public class KStreamImplTest {
|
|||
assertThat(exception.getMessage(), equalTo("named can't be null"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAllowNullProcessValuesSupplierOnProcess() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> testStream.processValues((FixedKeyProcessorSupplier<? super String, ? super String, Void>) null));
|
||||
assertThat(exception.getMessage(), equalTo("processorSupplier can't be null"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAllowNullProcessSupplierOnProcessValuesWithStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> testStream.processValues((FixedKeyProcessorSupplier<? super String, ? super String, Void>) null,
|
||||
"storeName"));
|
||||
assertThat(exception.getMessage(), equalTo("processorSupplier can't be null"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAllowNullProcessSupplierOnProcessValuesWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> testStream.process((ProcessorSupplier<? super String, ? super String, Void, Void>) null,
|
||||
Named.as("processor")));
|
||||
assertThat(exception.getMessage(), equalTo("processorSupplier can't be null"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAllowNullProcessSupplierOnProcessValuesWithNamedAndStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> testStream.process((ProcessorSupplier<? super String, ? super String, Void, Void>) null,
|
||||
Named.as("processor"), "stateStore"));
|
||||
assertThat(exception.getMessage(), equalTo("processorSupplier can't be null"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAllowNullStoreNamesOnProcessValues() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> testStream.processValues(fixedKeyProcessorSupplier, (String[]) null));
|
||||
assertThat(exception.getMessage(), equalTo("stateStoreNames can't be a null array"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAllowNullStoreNameOnProcessValues() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> testStream.processValues(fixedKeyProcessorSupplier, (String) null));
|
||||
assertThat(exception.getMessage(), equalTo("stateStoreNames can't be null"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAllowNullStoreNamesOnProcessValuesWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> testStream.processValues(fixedKeyProcessorSupplier, Named.as("processor"), (String[]) null));
|
||||
assertThat(exception.getMessage(), equalTo("stateStoreNames can't be a null array"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAllowNullStoreNameOnProcessValuesWithNamed() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> testStream.processValues(fixedKeyProcessorSupplier, Named.as("processor"), (String) null));
|
||||
assertThat(exception.getMessage(), equalTo("stateStoreNames can't be null"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAllowNullNamedOnProcessValues() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> testStream.processValues(fixedKeyProcessorSupplier, (Named) null));
|
||||
assertThat(exception.getMessage(), equalTo("named can't be null"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAllowNullNamedOnProcessValuesWithStores() {
|
||||
final NullPointerException exception = assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> testStream.processValues(fixedKeyProcessorSupplier, (Named) null, "storeName"));
|
||||
assertThat(exception.getMessage(), equalTo("named can't be null"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotMaterializedKTableFromKStream() {
|
||||
|
@ -2540,6 +2700,308 @@ public class KStreamImplTest {
|
|||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
@Test
|
||||
public void shouldProcessWithOldProcessorAndState() {
|
||||
final Consumed<String, String> consumed = Consumed.with(Serdes.String(), Serdes.String());
|
||||
|
||||
final StreamsBuilder builder = new StreamsBuilder();
|
||||
|
||||
final String input = "input";
|
||||
|
||||
builder.addStateStore(Stores.keyValueStoreBuilder(
|
||||
Stores.inMemoryKeyValueStore("sum"),
|
||||
Serdes.String(),
|
||||
Serdes.Integer()
|
||||
));
|
||||
|
||||
builder.stream(input, consumed)
|
||||
.process(() -> new org.apache.kafka.streams.processor.Processor<String, String>() {
|
||||
private KeyValueStore<String, Integer> sumStore;
|
||||
|
||||
@Override
|
||||
public void init(final ProcessorContext context) {
|
||||
this.sumStore = context.getStateStore("sum");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process(final String key, final String value) {
|
||||
final Integer counter = sumStore.get(key);
|
||||
if (counter == null) {
|
||||
sumStore.putIfAbsent(key, value.length());
|
||||
} else {
|
||||
if (value == null) {
|
||||
sumStore.delete(key);
|
||||
} else {
|
||||
sumStore.put(key, counter + value.length());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
||||
}, Named.as("p"), "sum");
|
||||
|
||||
final String topologyDescription = builder.build().describe().toString();
|
||||
|
||||
assertThat(
|
||||
topologyDescription,
|
||||
equalTo("Topologies:\n"
|
||||
+ " Sub-topology: 0\n"
|
||||
+ " Source: KSTREAM-SOURCE-0000000000 (topics: [input])\n"
|
||||
+ " --> p\n"
|
||||
+ " Processor: p (stores: [sum])\n"
|
||||
+ " --> none\n"
|
||||
+ " <-- KSTREAM-SOURCE-0000000000\n\n")
|
||||
);
|
||||
|
||||
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
|
||||
final TestInputTopic<String, String> inputTopic =
|
||||
driver.createInputTopic(
|
||||
input,
|
||||
Serdes.String().serializer(),
|
||||
Serdes.String().serializer()
|
||||
);
|
||||
|
||||
inputTopic.pipeInput("A", "0", 5L);
|
||||
inputTopic.pipeInput("B", "00", 100L);
|
||||
inputTopic.pipeInput("C", "000", 0L);
|
||||
inputTopic.pipeInput("D", "0000", 0L);
|
||||
inputTopic.pipeInput("A", "00000", 10L);
|
||||
inputTopic.pipeInput("A", "000000", 8L);
|
||||
|
||||
final KeyValueStore<String, Integer> sumStore = driver.getKeyValueStore("sum");
|
||||
assertEquals(12, sumStore.get("A").intValue());
|
||||
assertEquals(2, sumStore.get("B").intValue());
|
||||
assertEquals(3, sumStore.get("C").intValue());
|
||||
assertEquals(4, sumStore.get("D").intValue());
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("deprecation")
|
||||
@Test
|
||||
public void shouldBindStateWithOldProcessorSupplier() {
|
||||
final Consumed<String, String> consumed = Consumed.with(Serdes.String(), Serdes.String());
|
||||
|
||||
final StreamsBuilder builder = new StreamsBuilder();
|
||||
|
||||
final String input = "input";
|
||||
|
||||
builder.stream(input, consumed)
|
||||
.process(new org.apache.kafka.streams.processor.ProcessorSupplier<String, String>() {
|
||||
|
||||
@Override
|
||||
public org.apache.kafka.streams.processor.Processor<String, String> get() {
|
||||
return new org.apache.kafka.streams.processor.Processor<String, String>() {
|
||||
private KeyValueStore<String, Integer> sumStore;
|
||||
|
||||
@Override
|
||||
public void init(final ProcessorContext context) {
|
||||
this.sumStore = context.getStateStore("sum");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process(final String key, final String value) {
|
||||
final Integer counter = sumStore.get(key);
|
||||
if (counter == null) {
|
||||
sumStore.putIfAbsent(key, value.length());
|
||||
} else {
|
||||
if (value == null) {
|
||||
sumStore.delete(key);
|
||||
} else {
|
||||
sumStore.put(key, counter + value.length());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public Set<StoreBuilder<?>> stores() {
|
||||
final Set<StoreBuilder<?>> stores = new HashSet<>();
|
||||
stores.add(Stores.keyValueStoreBuilder(
|
||||
Stores.inMemoryKeyValueStore("sum"),
|
||||
Serdes.String(),
|
||||
Serdes.Integer()
|
||||
));
|
||||
return stores;
|
||||
}
|
||||
}, Named.as("p"));
|
||||
|
||||
final String topologyDescription = builder.build().describe().toString();
|
||||
|
||||
assertThat(
|
||||
topologyDescription,
|
||||
equalTo("Topologies:\n"
|
||||
+ " Sub-topology: 0\n"
|
||||
+ " Source: KSTREAM-SOURCE-0000000000 (topics: [input])\n"
|
||||
+ " --> p\n"
|
||||
+ " Processor: p (stores: [sum])\n"
|
||||
+ " --> none\n"
|
||||
+ " <-- KSTREAM-SOURCE-0000000000\n\n")
|
||||
);
|
||||
|
||||
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
|
||||
final TestInputTopic<String, String> inputTopic =
|
||||
driver.createInputTopic(
|
||||
input,
|
||||
Serdes.String().serializer(),
|
||||
Serdes.String().serializer()
|
||||
);
|
||||
|
||||
inputTopic.pipeInput("A", "0", 5L);
|
||||
inputTopic.pipeInput("B", "00", 100L);
|
||||
inputTopic.pipeInput("C", "000", 0L);
|
||||
inputTopic.pipeInput("D", "0000", 0L);
|
||||
inputTopic.pipeInput("A", "00000", 10L);
|
||||
inputTopic.pipeInput("A", "000000", 8L);
|
||||
|
||||
final KeyValueStore<String, Integer> sumStore = driver.getKeyValueStore("sum");
|
||||
assertEquals(12, sumStore.get("A").intValue());
|
||||
assertEquals(2, sumStore.get("B").intValue());
|
||||
assertEquals(3, sumStore.get("C").intValue());
|
||||
assertEquals(4, sumStore.get("D").intValue());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldBindStateWithOldProcessor() {
|
||||
final Consumed<String, String> consumed = Consumed.with(Serdes.String(), Serdes.String());
|
||||
|
||||
final StreamsBuilder builder = new StreamsBuilder();
|
||||
|
||||
final String input = "input";
|
||||
final String output = "output";
|
||||
|
||||
builder.stream(input, consumed)
|
||||
.process(() -> new ContextualProcessor<String, String, String, Integer>() {
|
||||
@Override
|
||||
public void process(final Record<String, String> record) {
|
||||
context().forward(record.withValue(record.value().length()));
|
||||
}
|
||||
}, Named.as("p"))
|
||||
.to(output, Produced.valueSerde(Serdes.Integer()));
|
||||
|
||||
final String topologyDescription = builder.build().describe().toString();
|
||||
|
||||
assertThat(
|
||||
topologyDescription,
|
||||
equalTo("Topologies:\n" +
|
||||
" Sub-topology: 0\n" +
|
||||
" Source: KSTREAM-SOURCE-0000000000 (topics: [input])\n" +
|
||||
" --> p\n" +
|
||||
" Processor: p (stores: [])\n" +
|
||||
" --> KSTREAM-SINK-0000000001\n" +
|
||||
" <-- KSTREAM-SOURCE-0000000000\n" +
|
||||
" Sink: KSTREAM-SINK-0000000001 (topic: output)\n" +
|
||||
" <-- p\n\n")
|
||||
);
|
||||
|
||||
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
|
||||
final TestInputTopic<String, String> inputTopic =
|
||||
driver.createInputTopic(
|
||||
input,
|
||||
Serdes.String().serializer(),
|
||||
Serdes.String().serializer()
|
||||
);
|
||||
final TestOutputTopic<String, Integer> outputTopic =
|
||||
driver.createOutputTopic(
|
||||
output,
|
||||
Serdes.String().deserializer(),
|
||||
Serdes.Integer().deserializer()
|
||||
);
|
||||
|
||||
inputTopic.pipeInput("A", "0", 5L);
|
||||
inputTopic.pipeInput("B", "00", 100L);
|
||||
inputTopic.pipeInput("C", "000", 0L);
|
||||
inputTopic.pipeInput("D", "0000", 0L);
|
||||
inputTopic.pipeInput("A", "00000", 10L);
|
||||
inputTopic.pipeInput("A", "000000", 8L);
|
||||
|
||||
final List<TestRecord<String, Integer>> outputExpectRecords = new ArrayList<>();
|
||||
outputExpectRecords.add(new TestRecord<>("A", 1, Instant.ofEpochMilli(5L)));
|
||||
outputExpectRecords.add(new TestRecord<>("B", 2, Instant.ofEpochMilli(100L)));
|
||||
outputExpectRecords.add(new TestRecord<>("C", 3, Instant.ofEpochMilli(0L)));
|
||||
outputExpectRecords.add(new TestRecord<>("D", 4, Instant.ofEpochMilli(0L)));
|
||||
outputExpectRecords.add(new TestRecord<>("A", 5, Instant.ofEpochMilli(10L)));
|
||||
outputExpectRecords.add(new TestRecord<>("A", 6, Instant.ofEpochMilli(8L)));
|
||||
|
||||
assertEquals(outputTopic.readRecordsToList(), outputExpectRecords);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldProcessValues() {
|
||||
final Consumed<String, String> consumed = Consumed.with(Serdes.String(), Serdes.String());
|
||||
|
||||
final StreamsBuilder builder = new StreamsBuilder();
|
||||
|
||||
final String input = "input";
|
||||
final String output = "output";
|
||||
|
||||
builder.stream(input, consumed)
|
||||
.processValues(() -> new ContextualFixedKeyProcessor<String, String, Integer>() {
|
||||
@Override
|
||||
public void process(final FixedKeyRecord<String, String> record) {
|
||||
context().forward(record.withValue(record.value().length()));
|
||||
}
|
||||
}, Named.as("fkp"))
|
||||
.to(output, Produced.valueSerde(Serdes.Integer()));
|
||||
|
||||
final String topologyDescription = builder.build().describe().toString();
|
||||
|
||||
assertThat(
|
||||
topologyDescription,
|
||||
equalTo("Topologies:\n" +
|
||||
" Sub-topology: 0\n" +
|
||||
" Source: KSTREAM-SOURCE-0000000000 (topics: [input])\n" +
|
||||
" --> fkp\n" +
|
||||
" Processor: fkp (stores: [])\n" +
|
||||
" --> KSTREAM-SINK-0000000001\n" +
|
||||
" <-- KSTREAM-SOURCE-0000000000\n" +
|
||||
" Sink: KSTREAM-SINK-0000000001 (topic: output)\n" +
|
||||
" <-- fkp\n\n")
|
||||
);
|
||||
|
||||
try (final TopologyTestDriver driver = new TopologyTestDriver(builder.build(), props)) {
|
||||
final TestInputTopic<String, String> inputTopic =
|
||||
driver.createInputTopic(
|
||||
input,
|
||||
Serdes.String().serializer(),
|
||||
Serdes.String().serializer()
|
||||
);
|
||||
final TestOutputTopic<String, Integer> outputTopic =
|
||||
driver.createOutputTopic(
|
||||
output,
|
||||
Serdes.String().deserializer(),
|
||||
Serdes.Integer().deserializer()
|
||||
);
|
||||
|
||||
inputTopic.pipeInput("A", "0", 5L);
|
||||
inputTopic.pipeInput("B", "00", 100L);
|
||||
inputTopic.pipeInput("C", "000", 0L);
|
||||
inputTopic.pipeInput("D", "0000", 0L);
|
||||
inputTopic.pipeInput("A", "00000", 10L);
|
||||
inputTopic.pipeInput("A", "000000", 8L);
|
||||
|
||||
final List<TestRecord<String, Integer>> outputExpectRecords = new ArrayList<>();
|
||||
outputExpectRecords.add(new TestRecord<>("A", 1, Instant.ofEpochMilli(5L)));
|
||||
outputExpectRecords.add(new TestRecord<>("B", 2, Instant.ofEpochMilli(100L)));
|
||||
outputExpectRecords.add(new TestRecord<>("C", 3, Instant.ofEpochMilli(0L)));
|
||||
outputExpectRecords.add(new TestRecord<>("D", 4, Instant.ofEpochMilli(0L)));
|
||||
outputExpectRecords.add(new TestRecord<>("A", 5, Instant.ofEpochMilli(10L)));
|
||||
outputExpectRecords.add(new TestRecord<>("A", 6, Instant.ofEpochMilli(8L)));
|
||||
|
||||
assertEquals(outputTopic.readRecordsToList(), outputExpectRecords);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldMaterializeKTableFromKStream() {
|
||||
final Consumed<String, String> consumed = Consumed.with(Serdes.String(), Serdes.String());
|
||||
|
|
|
@ -32,6 +32,7 @@ import org.apache.kafka.streams.processor.Punctuator;
|
|||
import org.apache.kafka.streams.processor.StateStore;
|
||||
import org.apache.kafka.streams.processor.TaskId;
|
||||
import org.apache.kafka.streams.processor.To;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyRecord;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
import org.apache.kafka.streams.query.Position;
|
||||
import org.apache.kafka.streams.state.RocksDBConfigSetter;
|
||||
|
@ -263,5 +264,18 @@ public class AbstractProcessorContextTest {
|
|||
public String changelogFor(final String storeName) {
|
||||
return ProcessorStateManager.storeChangelogTopic(applicationId(), storeName, taskId().topologyName());
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K, V> void forward(final FixedKeyRecord<K, V> record) {
|
||||
forward(new Record<>(record.key(), record.value(), record.timestamp(), record.headers()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K, V> void forward(final FixedKeyRecord<K, V> record, final String childName) {
|
||||
forward(
|
||||
new Record<>(record.key(), record.value(), record.timestamp(), record.headers()),
|
||||
childName
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.kafka.streams.kstream.Materialized;
|
|||
import org.apache.kafka.streams.processor.StateStore;
|
||||
import org.apache.kafka.streams.processor.TopicNameExtractor;
|
||||
import org.apache.kafka.streams.processor.api.Processor;
|
||||
import org.apache.kafka.streams.processor.api.ProcessorSupplier;
|
||||
import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder.SubtopologyDescription;
|
||||
import org.apache.kafka.streams.processor.internals.TopologyMetadata.Subtopology;
|
||||
import org.apache.kafka.streams.TopologyConfig;
|
||||
|
@ -734,32 +735,56 @@ public class InternalTopologyBuilderTest {
|
|||
|
||||
@Test
|
||||
public void shouldNotAllowNullNameWhenAddingSink() {
|
||||
assertThrows(NullPointerException.class, () -> builder.addSink(null, "topic", null, null, null));
|
||||
assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> builder.addSink(null, "topic", null, null, null)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAllowNullTopicWhenAddingSink() {
|
||||
assertThrows(NullPointerException.class, () -> builder.addSink("name", (String) null, null, null, null));
|
||||
assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> builder.addSink("name", (String) null, null, null, null)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAllowNullTopicChooserWhenAddingSink() {
|
||||
assertThrows(NullPointerException.class, () -> builder.addSink("name", (TopicNameExtractor<Object, Object>) null, null, null, null));
|
||||
assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> builder.addSink("name", (TopicNameExtractor<Object, Object>) null, null, null, null)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAllowNullNameWhenAddingProcessor() {
|
||||
assertThrows(NullPointerException.class, () -> builder.addProcessor(null, () -> null));
|
||||
assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> builder.addProcessor(
|
||||
null,
|
||||
(ProcessorSupplier<Object, Object, Object, Object>) () -> null
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAllowNullProcessorSupplier() {
|
||||
assertThrows(NullPointerException.class, () -> builder.addProcessor("name", null));
|
||||
assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> builder.addProcessor(
|
||||
"name",
|
||||
(ProcessorSupplier<Object, Object, Object, Object>) null
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void shouldNotAllowNullNameWhenAddingSource() {
|
||||
assertThrows(NullPointerException.class, () -> builder.addSource(null, null, null, null, null, Pattern.compile(".*")));
|
||||
assertThrows(
|
||||
NullPointerException.class,
|
||||
() -> builder.addSource(null, null, null, null, null, Pattern.compile(".*"))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -1396,9 +1396,10 @@ public class StreamThreadTest {
|
|||
internalTopologyBuilder.addSource(null, "name", null, null, null, topic1);
|
||||
final AtomicBoolean shouldThrow = new AtomicBoolean(false);
|
||||
final AtomicBoolean processed = new AtomicBoolean(false);
|
||||
// TODO check if needs to be extended
|
||||
internalTopologyBuilder.addProcessor(
|
||||
"proc",
|
||||
() -> record -> {
|
||||
(ProcessorSupplier<Object, Object, Object, Object>) () -> record -> {
|
||||
if (shouldThrow.get()) {
|
||||
throw new TaskCorruptedException(singleton(task1));
|
||||
} else {
|
||||
|
@ -1787,6 +1788,7 @@ public class StreamThreadTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldPunctuateActiveTask() {
|
||||
final List<Long> punctuatedStreamTime = new ArrayList<>();
|
||||
final List<Long> punctuatedWallClockTime = new ArrayList<>();
|
||||
|
@ -1857,6 +1859,7 @@ public class StreamThreadTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldPunctuateWithTimestampPreservedInProcessorContext() {
|
||||
final org.apache.kafka.streams.kstream.TransformerSupplier<Object, Object, KeyValue<Object, Object>> punctuateProcessor =
|
||||
() -> new org.apache.kafka.streams.kstream.Transformer<Object, Object, KeyValue<Object, Object>>() {
|
||||
|
|
|
@ -145,6 +145,7 @@ public class CachingPersistentWindowStoreTest {
|
|||
}
|
||||
|
||||
@Test
|
||||
@SuppressWarnings("deprecation")
|
||||
public void shouldNotReturnDuplicatesInRanges() {
|
||||
final StreamsBuilder builder = new StreamsBuilder();
|
||||
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.kafka.streams.processor.StateRestoreCallback;
|
|||
import org.apache.kafka.streams.processor.StateStore;
|
||||
import org.apache.kafka.streams.processor.TaskId;
|
||||
import org.apache.kafka.streams.processor.To;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyRecord;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
import org.apache.kafka.streams.processor.internals.AbstractProcessorContext;
|
||||
import org.apache.kafka.streams.processor.internals.ChangelogRecordDeserializationHelper;
|
||||
|
@ -500,4 +501,18 @@ public class InternalMockProcessorContext<KOut, VOut>
|
|||
public String changelogFor(final String storeName) {
|
||||
return storeToChangelogTopic.get(storeName);
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K extends KOut, V extends VOut> void forward(final FixedKeyRecord<K, V> record) {
|
||||
forward(new Record<>(record.key(), record.value(), record.timestamp(), record.headers()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K extends KOut, V extends VOut> void forward(final FixedKeyRecord<K, V> record,
|
||||
final String childName) {
|
||||
forward(
|
||||
new Record<>(record.key(), record.value(), record.timestamp(), record.headers()),
|
||||
childName
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,164 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.test;
|
||||
|
||||
import static org.hamcrest.MatcherAssert.assertThat;
|
||||
import static org.hamcrest.Matchers.is;
|
||||
|
||||
import java.time.Duration;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.kafka.streams.KeyValueTimestamp;
|
||||
import org.apache.kafka.streams.processor.Cancellable;
|
||||
import org.apache.kafka.streams.processor.PunctuationType;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessor;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessorContext;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyRecord;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
import org.apache.kafka.streams.state.ValueAndTimestamp;
|
||||
|
||||
public class MockApiFixedKeyProcessor<KIn, VIn, VOut> implements FixedKeyProcessor<KIn, VIn, VOut> {
|
||||
|
||||
private final ArrayList<FixedKeyRecord<KIn, VIn>> processed = new ArrayList<>();
|
||||
private final Map<KIn, ValueAndTimestamp<VIn>> lastValueAndTimestampPerKey = new HashMap<>();
|
||||
|
||||
private final ArrayList<Long> punctuatedStreamTime = new ArrayList<>();
|
||||
private final ArrayList<Long> punctuatedSystemTime = new ArrayList<>();
|
||||
|
||||
private Cancellable scheduleCancellable;
|
||||
|
||||
private final PunctuationType punctuationType;
|
||||
private final long scheduleInterval;
|
||||
|
||||
private boolean commitRequested = false;
|
||||
private FixedKeyProcessorContext<KIn, VOut> context;
|
||||
|
||||
public MockApiFixedKeyProcessor(final PunctuationType punctuationType,
|
||||
final long scheduleInterval) {
|
||||
this.punctuationType = punctuationType;
|
||||
this.scheduleInterval = scheduleInterval;
|
||||
}
|
||||
|
||||
public MockApiFixedKeyProcessor() {
|
||||
this(PunctuationType.STREAM_TIME, -1);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void init(final FixedKeyProcessorContext<KIn, VOut> context) {
|
||||
this.context = context;
|
||||
if (scheduleInterval > 0L) {
|
||||
scheduleCancellable = context.schedule(
|
||||
Duration.ofMillis(scheduleInterval),
|
||||
punctuationType,
|
||||
(punctuationType == PunctuationType.STREAM_TIME ? punctuatedStreamTime : punctuatedSystemTime)::add
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void process(final FixedKeyRecord<KIn, VIn> record) {
|
||||
final KIn key = record.key();
|
||||
final VIn value = record.value();
|
||||
final KeyValueTimestamp<KIn, VIn> keyValueTimestamp = new KeyValueTimestamp<>(key, value, record.timestamp());
|
||||
|
||||
if (value != null) {
|
||||
lastValueAndTimestampPerKey.put(key, ValueAndTimestamp.make(value, record.timestamp()));
|
||||
} else {
|
||||
lastValueAndTimestampPerKey.remove(key);
|
||||
}
|
||||
|
||||
processed.add(record);
|
||||
|
||||
if (commitRequested) {
|
||||
context.commit();
|
||||
commitRequested = false;
|
||||
}
|
||||
}
|
||||
|
||||
public void checkAndClearProcessResult(final KeyValueTimestamp<?, ?>... expected) {
|
||||
assertThat("the number of outputs:" + processed, processed.size(), is(expected.length));
|
||||
for (int i = 0; i < expected.length; i++) {
|
||||
final FixedKeyRecord<KIn, VIn> record = processed.get(i);
|
||||
assertThat(
|
||||
"output[" + i + "]:",
|
||||
new KeyValueTimestamp<>(record.key(), record.value(), record.timestamp()),
|
||||
is(expected[i])
|
||||
);
|
||||
}
|
||||
|
||||
processed.clear();
|
||||
}
|
||||
|
||||
public void checkAndClearProcessedRecords(final Record<?, ?>... expected) {
|
||||
assertThat("the number of outputs:" + processed, processed.size(), is(expected.length));
|
||||
for (int i = 0; i < expected.length; i++) {
|
||||
assertThat("output[" + i + "]:", processed.get(i), is(expected[i]));
|
||||
}
|
||||
|
||||
processed.clear();
|
||||
}
|
||||
|
||||
public void requestCommit() {
|
||||
commitRequested = true;
|
||||
}
|
||||
|
||||
public void checkEmptyAndClearProcessResult() {
|
||||
assertThat("the number of outputs:", processed.size(), is(0));
|
||||
processed.clear();
|
||||
}
|
||||
|
||||
public void checkAndClearPunctuateResult(final PunctuationType type, final long... expected) {
|
||||
final ArrayList<Long> punctuated = type == PunctuationType.STREAM_TIME ? punctuatedStreamTime : punctuatedSystemTime;
|
||||
assertThat("the number of outputs:", punctuated.size(), is(expected.length));
|
||||
|
||||
for (int i = 0; i < expected.length; i++) {
|
||||
assertThat("output[" + i + "]:", punctuated.get(i), is(expected[i]));
|
||||
}
|
||||
|
||||
processed.clear();
|
||||
}
|
||||
|
||||
public ArrayList<KeyValueTimestamp<KIn, VIn>> processed() {
|
||||
return processed
|
||||
.stream()
|
||||
.map(r -> new KeyValueTimestamp<>(r.key(), r.value(), r.timestamp()))
|
||||
.collect(Collectors.toCollection(ArrayList::new));
|
||||
}
|
||||
|
||||
public Map<KIn, ValueAndTimestamp<VIn>> lastValueAndTimestampPerKey() {
|
||||
return lastValueAndTimestampPerKey;
|
||||
}
|
||||
|
||||
public List<Long> punctuatedStreamTime() {
|
||||
return punctuatedStreamTime;
|
||||
}
|
||||
|
||||
public Cancellable scheduleCancellable() {
|
||||
return scheduleCancellable;
|
||||
}
|
||||
|
||||
public FixedKeyProcessorContext<KIn, VOut> context() {
|
||||
return context;
|
||||
}
|
||||
|
||||
public void context(final FixedKeyProcessorContext<KIn, VOut> context) {
|
||||
this.context = context;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,74 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.kafka.test;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import org.apache.kafka.streams.processor.PunctuationType;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessor;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyProcessorSupplier;
|
||||
|
||||
public class MockApiFixedKeyProcessorSupplier<KIn, VIn, VOut>
|
||||
implements FixedKeyProcessorSupplier<KIn, VIn, VOut> {
|
||||
|
||||
private final long scheduleInterval;
|
||||
private final PunctuationType punctuationType;
|
||||
private final List<MockApiFixedKeyProcessor<KIn, VIn, VOut>> processors = new ArrayList<>();
|
||||
|
||||
public MockApiFixedKeyProcessorSupplier() {
|
||||
this(-1L);
|
||||
}
|
||||
|
||||
public MockApiFixedKeyProcessorSupplier(final long scheduleInterval) {
|
||||
this(scheduleInterval, PunctuationType.STREAM_TIME);
|
||||
}
|
||||
|
||||
public MockApiFixedKeyProcessorSupplier(final long scheduleInterval, final PunctuationType punctuationType) {
|
||||
this.scheduleInterval = scheduleInterval;
|
||||
this.punctuationType = punctuationType;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FixedKeyProcessor<KIn, VIn, VOut> get() {
|
||||
final MockApiFixedKeyProcessor<KIn, VIn, VOut> processor = new MockApiFixedKeyProcessor<>(punctuationType, scheduleInterval);
|
||||
|
||||
// to keep tests simple, ignore calls from ApiUtils.checkSupplier
|
||||
if (!StreamsTestUtils.isCheckSupplierCall()) {
|
||||
processors.add(processor);
|
||||
}
|
||||
|
||||
return processor;
|
||||
}
|
||||
|
||||
// get the captured processor assuming that only one processor gets returned from this supplier
|
||||
public MockApiFixedKeyProcessor<KIn, VIn, VOut> theCapturedProcessor() {
|
||||
return capturedProcessors(1).get(0);
|
||||
}
|
||||
|
||||
public int capturedProcessorsCount() {
|
||||
return processors.size();
|
||||
}
|
||||
|
||||
// get the captured processors with the expected number
|
||||
public List<MockApiFixedKeyProcessor<KIn, VIn, VOut>> capturedProcessors(final int expectedNumberOfProcessors) {
|
||||
assertEquals(expectedNumberOfProcessors, processors.size());
|
||||
|
||||
return processors;
|
||||
}
|
||||
}
|
|
@ -22,6 +22,7 @@ import org.apache.kafka.streams.processor.PunctuationType;
|
|||
import org.apache.kafka.streams.processor.api.Processor;
|
||||
import org.apache.kafka.streams.processor.api.ProcessorContext;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
|
||||
import org.apache.kafka.streams.state.ValueAndTimestamp;
|
||||
|
||||
import java.time.Duration;
|
||||
|
@ -135,6 +136,12 @@ public class MockApiProcessor<KIn, VIn, KOut, VOut> implements Processor<KIn, VI
|
|||
processed.clear();
|
||||
}
|
||||
|
||||
public void addProcessorMetadata(final String key, final long value) {
|
||||
if (context instanceof InternalProcessorContext) {
|
||||
((InternalProcessorContext<KOut, VOut>) context).addProcessorMetadataKeyValue(key, value);
|
||||
}
|
||||
}
|
||||
|
||||
public ArrayList<KeyValueTimestamp<KIn, VIn>> processed() {
|
||||
return processed
|
||||
.stream()
|
||||
|
|
|
@ -25,7 +25,9 @@ import org.apache.kafka.streams.processor.StateRestoreCallback;
|
|||
import org.apache.kafka.streams.processor.StateStore;
|
||||
import org.apache.kafka.streams.processor.TaskId;
|
||||
import org.apache.kafka.streams.processor.To;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyRecord;
|
||||
import org.apache.kafka.streams.processor.api.MockProcessorContext;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
|
||||
import org.apache.kafka.streams.processor.internals.ProcessorMetadata;
|
||||
import org.apache.kafka.streams.processor.internals.ProcessorNode;
|
||||
|
@ -232,4 +234,18 @@ public class MockInternalNewProcessorContext<KOut, VOut> extends MockProcessorCo
|
|||
public ProcessorMetadata getProcessorMetadata() {
|
||||
return processorMetadata;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K extends KOut, V extends VOut> void forward(final FixedKeyRecord<K, V> record) {
|
||||
forward(new Record<>(record.key(), record.value(), record.timestamp(), record.headers()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K extends KOut, V extends VOut> void forward(final FixedKeyRecord<K, V> record,
|
||||
final String childName) {
|
||||
forward(
|
||||
new Record<>(record.key(), record.value(), record.timestamp(), record.headers()),
|
||||
childName
|
||||
);
|
||||
}
|
||||
}
|
|
@ -24,6 +24,7 @@ import org.apache.kafka.streams.processor.StateRestoreCallback;
|
|||
import org.apache.kafka.streams.processor.StateStore;
|
||||
import org.apache.kafka.streams.processor.TaskId;
|
||||
import org.apache.kafka.streams.processor.To;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyRecord;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
import org.apache.kafka.streams.processor.api.RecordMetadata;
|
||||
import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
|
||||
|
@ -207,4 +208,17 @@ public class MockInternalProcessorContext extends MockProcessorContext implement
|
|||
public ProcessorMetadata getProcessorMetadata() {
|
||||
return processorMetadata;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K, V> void forward(final FixedKeyRecord<K, V> record) {
|
||||
forward(new Record<>(record.key(), record.value(), record.timestamp(), record.headers()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K, V> void forward(final FixedKeyRecord<K, V> record, final String childName) {
|
||||
forward(
|
||||
new Record<>(record.key(), record.value(), record.timestamp(), record.headers()),
|
||||
childName
|
||||
);
|
||||
}
|
||||
}
|
|
@ -16,14 +16,14 @@
|
|||
*/
|
||||
package org.apache.kafka.test;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import org.apache.kafka.streams.processor.PunctuationType;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
|
||||
import org.apache.kafka.streams.processor.internals.ProcessorAdapter;
|
||||
import org.apache.kafka.streams.processor.internals.ProcessorNode;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
public class MockProcessorNode<KIn, VIn, KOut, VOut> extends ProcessorNode<KIn, VIn, KOut, VOut> {
|
||||
|
||||
private static final String NAME = "MOCK-PROCESS-";
|
||||
|
@ -46,8 +46,9 @@ public class MockProcessorNode<KIn, VIn, KOut, VOut> extends ProcessorNode<KIn,
|
|||
this(new MockProcessor<>());
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
private MockProcessorNode(final MockProcessor<KIn, VIn> mockProcessor) {
|
||||
super(NAME + INDEX.getAndIncrement(), mockProcessor, Collections.<String>emptySet());
|
||||
super(NAME + INDEX.getAndIncrement(), ProcessorAdapter.adapt(mockProcessor), Collections.<String>emptySet());
|
||||
|
||||
this.mockProcessor = mockProcessor;
|
||||
}
|
||||
|
@ -60,7 +61,7 @@ public class MockProcessorNode<KIn, VIn, KOut, VOut> extends ProcessorNode<KIn,
|
|||
|
||||
@Override
|
||||
public void process(final Record<KIn, VIn> record) {
|
||||
processor().process(record);
|
||||
mockProcessor.process(record.key(), record.value());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -27,6 +27,7 @@ import org.apache.kafka.streams.processor.StateRestoreCallback;
|
|||
import org.apache.kafka.streams.processor.StateStore;
|
||||
import org.apache.kafka.streams.processor.TaskId;
|
||||
import org.apache.kafka.streams.processor.To;
|
||||
import org.apache.kafka.streams.processor.api.FixedKeyRecord;
|
||||
import org.apache.kafka.streams.processor.api.Record;
|
||||
import org.apache.kafka.streams.processor.internals.AbstractProcessorContext;
|
||||
import org.apache.kafka.streams.processor.internals.MockStreamsMetrics;
|
||||
|
@ -152,4 +153,17 @@ public class NoOpProcessorContext extends AbstractProcessorContext<Object, Objec
|
|||
public String changelogFor(final String storeName) {
|
||||
return ProcessorStateManager.storeChangelogTopic(applicationId(), storeName, taskId().topologyName());
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K, V> void forward(final FixedKeyRecord<K, V> record) {
|
||||
forward(new Record<>(record.key(), record.value(), record.timestamp(), record.headers()));
|
||||
}
|
||||
|
||||
@Override
|
||||
public <K, V> void forward(final FixedKeyRecord<K, V> record, final String childName) {
|
||||
forward(
|
||||
new Record<>(record.key(), record.value(), record.timestamp(), record.headers()),
|
||||
childName
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -28,7 +28,7 @@ import org.apache.kafka.streams.kstream.{
|
|||
KStream => KStreamJ
|
||||
}
|
||||
import org.apache.kafka.streams.processor.TopicNameExtractor
|
||||
import org.apache.kafka.streams.processor.api.ProcessorSupplier
|
||||
import org.apache.kafka.streams.processor.api.{FixedKeyProcessorSupplier, ProcessorSupplier}
|
||||
import org.apache.kafka.streams.scala.FunctionsCompatConversions.{
|
||||
FlatValueMapperFromFunction,
|
||||
FlatValueMapperWithKeyFromFunction,
|
||||
|
@ -558,6 +558,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @return a [[KStream]] that contains more or less records with new key and value (possibly of different type)
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#transform`
|
||||
*/
|
||||
@deprecated(since = "3.3", message = "Use process(ProcessorSupplier, String*) instead.")
|
||||
def transform[K1, V1](
|
||||
transformerSupplier: TransformerSupplier[K, V, KeyValue[K1, V1]],
|
||||
stateStoreNames: String*
|
||||
|
@ -580,6 +581,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @return a [[KStream]] that contains more or less records with new key and value (possibly of different type)
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#transform`
|
||||
*/
|
||||
@deprecated(since = "3.3", message = "Use process(ProcessorSupplier, Named, String*) instead.")
|
||||
def transform[K1, V1](
|
||||
transformerSupplier: TransformerSupplier[K, V, KeyValue[K1, V1]],
|
||||
named: Named,
|
||||
|
@ -602,6 +604,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @return a [[KStream]] that contains more or less records with new key and value (possibly of different type)
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#transform`
|
||||
*/
|
||||
@deprecated(since = "3.3", message = "Use process(ProcessorSupplier, String*) instead.")
|
||||
def flatTransform[K1, V1](
|
||||
transformerSupplier: TransformerSupplier[K, V, Iterable[KeyValue[K1, V1]]],
|
||||
stateStoreNames: String*
|
||||
|
@ -624,6 +627,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @return a [[KStream]] that contains more or less records with new key and value (possibly of different type)
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#transform`
|
||||
*/
|
||||
@deprecated(since = "3.3", message = "Use process(ProcessorSupplier, Named, String*) instead.")
|
||||
def flatTransform[K1, V1](
|
||||
transformerSupplier: TransformerSupplier[K, V, Iterable[KeyValue[K1, V1]]],
|
||||
named: Named,
|
||||
|
@ -646,6 +650,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#transformValues`
|
||||
*/
|
||||
@deprecated(since = "3.3", message = "Use processValues(FixedKeyProcessorSupplier, Named, String*) instead.")
|
||||
def flatTransformValues[VR](
|
||||
valueTransformerSupplier: ValueTransformerSupplier[V, Iterable[VR]],
|
||||
stateStoreNames: String*
|
||||
|
@ -668,6 +673,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#transformValues`
|
||||
*/
|
||||
@deprecated(since = "3.3", message = "Use processValues(FixedKeyProcessorSupplier, Named, String*) instead.")
|
||||
def flatTransformValues[VR](
|
||||
valueTransformerSupplier: ValueTransformerSupplier[V, Iterable[VR]],
|
||||
named: Named,
|
||||
|
@ -690,6 +696,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#transformValues`
|
||||
*/
|
||||
@deprecated(since = "3.3", message = "Use processValues(FixedKeyProcessorSupplier, String*) instead.")
|
||||
def flatTransformValues[VR](
|
||||
valueTransformerSupplier: ValueTransformerWithKeySupplier[K, V, Iterable[VR]],
|
||||
stateStoreNames: String*
|
||||
|
@ -712,6 +719,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#transformValues`
|
||||
*/
|
||||
@deprecated(since = "3.3", message = "Use processValues(FixedKeyProcessorSupplier, Named, String*) instead.")
|
||||
def flatTransformValues[VR](
|
||||
valueTransformerSupplier: ValueTransformerWithKeySupplier[K, V, Iterable[VR]],
|
||||
named: Named,
|
||||
|
@ -733,6 +741,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#transformValues`
|
||||
*/
|
||||
@deprecated(since = "3.3", message = "Use processValues(FixedKeyProcessorSupplier, String*) instead.")
|
||||
def transformValues[VR](
|
||||
valueTransformerSupplier: ValueTransformerSupplier[V, VR],
|
||||
stateStoreNames: String*
|
||||
|
@ -754,6 +763,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#transformValues`
|
||||
*/
|
||||
@deprecated(since = "3.3", message = "Use processValues(FixedKeyProcessorSupplier, Named, String*) instead.")
|
||||
def transformValues[VR](
|
||||
valueTransformerSupplier: ValueTransformerSupplier[V, VR],
|
||||
named: Named,
|
||||
|
@ -775,6 +785,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#transformValues`
|
||||
*/
|
||||
@deprecated(since = "3.3", message = "Use processValues(FixedKeyProcessorSupplier, String*) instead.")
|
||||
def transformValues[VR](
|
||||
valueTransformerSupplier: ValueTransformerWithKeySupplier[K, V, VR],
|
||||
stateStoreNames: String*
|
||||
|
@ -796,6 +807,7 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @return a [[KStream]] that contains records with unmodified key and new values (possibly of different type)
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#transformValues`
|
||||
*/
|
||||
@deprecated(since = "3.3", message = "Use processValues(FixedKeyProcessorSupplier, Named, String*) instead.")
|
||||
def transformValues[VR](
|
||||
valueTransformerSupplier: ValueTransformerWithKeySupplier[K, V, VR],
|
||||
named: Named,
|
||||
|
@ -824,24 +836,6 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
inner.process(processorSupplierJ, stateStoreNames: _*)
|
||||
}
|
||||
|
||||
/**
|
||||
* Process all records in this stream, one record at a time, by applying a `Processor` (provided by the given
|
||||
* `processorSupplier`).
|
||||
* In order to assign a state, the state must be created and added via `addStateStore` before they can be connected
|
||||
* to the `Processor`.
|
||||
* It's not required to connect global state stores that are added via `addGlobalStore`;
|
||||
* read-only access to global state stores is available by default.
|
||||
*
|
||||
* Note that this overload takes a ProcessorSupplier instead of a Function to avoid post-erasure ambiguity with
|
||||
* the older (deprecated) overload.
|
||||
*
|
||||
* @param processorSupplier a supplier for [[org.apache.kafka.streams.processor.api.Processor]]
|
||||
* @param stateStoreNames the names of the state store used by the processor
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#process`
|
||||
*/
|
||||
def process(processorSupplier: ProcessorSupplier[K, V, Void, Void], stateStoreNames: String*): Unit =
|
||||
inner.process(processorSupplier, stateStoreNames: _*)
|
||||
|
||||
/**
|
||||
* Process all records in this stream, one record at a time, by applying a `Processor` (provided by the given
|
||||
* `processorSupplier`).
|
||||
|
@ -865,6 +859,24 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
inner.process(processorSupplierJ, named, stateStoreNames: _*)
|
||||
}
|
||||
|
||||
/**
|
||||
* Process all records in this stream, one record at a time, by applying a `Processor` (provided by the given
|
||||
* `processorSupplier`).
|
||||
* In order to assign a state, the state must be created and added via `addStateStore` before they can be connected
|
||||
* to the `Processor`.
|
||||
* It's not required to connect global state stores that are added via `addGlobalStore`;
|
||||
* read-only access to global state stores is available by default.
|
||||
*
|
||||
* Note that this overload takes a ProcessorSupplier instead of a Function to avoid post-erasure ambiguity with
|
||||
* the older (deprecated) overload.
|
||||
*
|
||||
* @param processorSupplier a supplier for [[org.apache.kafka.streams.processor.api.Processor]]
|
||||
* @param stateStoreNames the names of the state store used by the processor
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#process`
|
||||
*/
|
||||
def process[KR, VR](processorSupplier: ProcessorSupplier[K, V, KR, VR], stateStoreNames: String*): KStream[KR, VR] =
|
||||
new KStream(inner.process(processorSupplier, stateStoreNames: _*))
|
||||
|
||||
/**
|
||||
* Process all records in this stream, one record at a time, by applying a `Processor` (provided by the given
|
||||
* `processorSupplier`).
|
||||
|
@ -881,8 +893,56 @@ class KStream[K, V](val inner: KStreamJ[K, V]) {
|
|||
* @param stateStoreNames the names of the state store used by the processor
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#process`
|
||||
*/
|
||||
def process(processorSupplier: ProcessorSupplier[K, V, Void, Void], named: Named, stateStoreNames: String*): Unit =
|
||||
inner.process(processorSupplier, named, stateStoreNames: _*)
|
||||
def process[KR, VR](
|
||||
processorSupplier: ProcessorSupplier[K, V, KR, VR],
|
||||
named: Named,
|
||||
stateStoreNames: String*
|
||||
): KStream[KR, VR] =
|
||||
new KStream(inner.process(processorSupplier, named, stateStoreNames: _*))
|
||||
|
||||
/**
|
||||
* Process all records in this stream, one record at a time, by applying a `FixedKeyProcessor` (provided by the given
|
||||
* `processorSupplier`).
|
||||
* In order to assign a state, the state must be created and added via `addStateStore` before they can be connected
|
||||
* to the `FixedKeyProcessor`.
|
||||
* It's not required to connect global state stores that are added via `addGlobalStore`;
|
||||
* read-only access to global state stores is available by default.
|
||||
*
|
||||
* Note that this overload takes a FixedKeyProcessorSupplier instead of a Function to avoid post-erasure ambiguity with
|
||||
* the older (deprecated) overload.
|
||||
*
|
||||
* @param processorSupplier a supplier for [[org.apache.kafka.streams.processor.api.FixedKeyProcessor]]
|
||||
* @param stateStoreNames the names of the state store used by the processor
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#process`
|
||||
*/
|
||||
def processValues[VR](
|
||||
processorSupplier: FixedKeyProcessorSupplier[K, V, VR],
|
||||
stateStoreNames: String*
|
||||
): KStream[K, VR] =
|
||||
new KStream(inner.processValues(processorSupplier, stateStoreNames: _*))
|
||||
|
||||
/**
|
||||
* Process all records in this stream, one record at a time, by applying a `FixedKeyProcessor` (provided by the given
|
||||
* `processorSupplier`).
|
||||
* In order to assign a state, the state must be created and added via `addStateStore` before they can be connected
|
||||
* to the `FixedKeyProcessor`.
|
||||
* It's not required to connect global state stores that are added via `addGlobalStore`;
|
||||
* read-only access to global state stores is available by default.
|
||||
*
|
||||
* Note that this overload takes a ProcessorSupplier instead of a Function to avoid post-erasure ambiguity with
|
||||
* the older (deprecated) overload.
|
||||
*
|
||||
* @param processorSupplier a supplier for [[org.apache.kafka.streams.processor.api.FixedKeyProcessor]]
|
||||
* @param named a [[Named]] config used to name the processor in the topology
|
||||
* @param stateStoreNames the names of the state store used by the processor
|
||||
* @see `org.apache.kafka.streams.kstream.KStream#process`
|
||||
*/
|
||||
def processValues[VR](
|
||||
processorSupplier: FixedKeyProcessorSupplier[K, V, VR],
|
||||
named: Named,
|
||||
stateStoreNames: String*
|
||||
): KStream[K, VR] =
|
||||
new KStream(inner.processValues(processorSupplier, named, stateStoreNames: _*))
|
||||
|
||||
/**
|
||||
* Group the records by their current key into a [[KGroupedStream]]
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.apache.kafka.streams.{KeyValue, StreamsConfig, TopologyDescription, S
|
|||
import org.junit.jupiter.api.Assertions._
|
||||
import org.junit.jupiter.api._
|
||||
|
||||
import scala.annotation.nowarn
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
/**
|
||||
|
@ -275,6 +276,7 @@ class TopologyTest {
|
|||
assertEquals(getTopologyScala, getTopologyJava)
|
||||
}
|
||||
|
||||
@nowarn
|
||||
@Test
|
||||
def shouldBuildIdenticalTopologyInJavaNScalaTransform(): Unit = {
|
||||
|
||||
|
@ -301,6 +303,7 @@ class TopologyTest {
|
|||
streamBuilder.build().describe()
|
||||
}
|
||||
|
||||
@nowarn
|
||||
// build the Java topology
|
||||
def getTopologyJava: TopologyDescription = {
|
||||
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.kafka.streams.scala.utils.TestDriver
|
|||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
import scala.annotation.nowarn
|
||||
import scala.jdk.CollectionConverters._
|
||||
|
||||
class KStreamTest extends TestDriver {
|
||||
|
@ -221,6 +222,7 @@ class KStreamTest extends TestDriver {
|
|||
testDriver.close()
|
||||
}
|
||||
|
||||
@nowarn
|
||||
@Test
|
||||
def testTransformCorrectlyRecords(): Unit = {
|
||||
class TestTransformer extends Transformer[String, String, KeyValue[String, String]] {
|
||||
|
@ -256,6 +258,7 @@ class KStreamTest extends TestDriver {
|
|||
testDriver.close()
|
||||
}
|
||||
|
||||
@nowarn
|
||||
@Test
|
||||
def testFlatTransformCorrectlyRecords(): Unit = {
|
||||
class TestTransformer extends Transformer[String, String, Iterable[KeyValue[String, String]]] {
|
||||
|
@ -291,6 +294,7 @@ class KStreamTest extends TestDriver {
|
|||
testDriver.close()
|
||||
}
|
||||
|
||||
@nowarn
|
||||
@Test
|
||||
def testCorrectlyFlatTransformValuesInRecords(): Unit = {
|
||||
class TestTransformer extends ValueTransformer[String, Iterable[String]] {
|
||||
|
@ -327,6 +331,7 @@ class KStreamTest extends TestDriver {
|
|||
testDriver.close()
|
||||
}
|
||||
|
||||
@nowarn
|
||||
@Test
|
||||
def testCorrectlyFlatTransformValuesInRecordsWithKey(): Unit = {
|
||||
class TestTransformer extends ValueTransformerWithKey[String, String, Iterable[String]] {
|
||||
|
@ -443,6 +448,7 @@ class KStreamTest extends TestDriver {
|
|||
assertEquals("my-name", joinNode.name())
|
||||
}
|
||||
|
||||
@nowarn
|
||||
@Test
|
||||
def testSettingNameOnTransform(): Unit = {
|
||||
class TestTransformer extends Transformer[String, String, KeyValue[String, String]] {
|
||||
|
|
|
@ -76,6 +76,8 @@ public class MockProcessorContext<KForward, VForward> implements ProcessorContex
|
|||
|
||||
// settable record metadata ================================================
|
||||
private MockRecordMetadata recordMetadata;
|
||||
private Long currentSystemTimeMs;
|
||||
private Long currentStreamTimeMs;
|
||||
|
||||
// mocks ================================================
|
||||
private final Map<String, StateStore> stateStores = new HashMap<>();
|
||||
|
@ -286,12 +288,18 @@ public class MockProcessorContext<KForward, VForward> implements ProcessorContex
|
|||
|
||||
@Override
|
||||
public long currentSystemTimeMs() {
|
||||
throw new UnsupportedOperationException();
|
||||
if (currentSystemTimeMs == null) {
|
||||
throw new IllegalStateException("System time must be set before use via setCurrentSystemTimeMs().");
|
||||
}
|
||||
return currentSystemTimeMs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public long currentStreamTimeMs() {
|
||||
throw new UnsupportedOperationException();
|
||||
if (currentStreamTimeMs == null) {
|
||||
throw new IllegalStateException("Stream time must be set before use via setCurrentStreamTimeMs().");
|
||||
}
|
||||
return currentStreamTimeMs;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -336,6 +344,14 @@ public class MockProcessorContext<KForward, VForward> implements ProcessorContex
|
|||
recordMetadata = new MockRecordMetadata(topic, partition, offset);
|
||||
}
|
||||
|
||||
public void setCurrentSystemTimeMs(final long currentSystemTimeMs) {
|
||||
this.currentSystemTimeMs = currentSystemTimeMs;
|
||||
}
|
||||
|
||||
public void setCurrentStreamTimeMs(final long currentStreamTimeMs) {
|
||||
this.currentStreamTimeMs = currentStreamTimeMs;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<RecordMetadata> recordMetadata() {
|
||||
return Optional.ofNullable(recordMetadata);
|
||||
|
|
Loading…
Reference in New Issue