diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index bf12927bda45c..086ac81083259 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -48,11 +48,11 @@ import org.apache.kafka.streams.errors.UnknownStateStoreException; import org.apache.kafka.streams.errors.InvalidStateStorePartitionException; import org.apache.kafka.streams.internals.metrics.ClientMetrics; -import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.ThreadMetadata; +import org.apache.kafka.streams.processor.api.Processor; import org.apache.kafka.streams.processor.internals.ClientUtils; import org.apache.kafka.streams.processor.internals.DefaultKafkaClientSupplier; import org.apache.kafka.streams.processor.internals.GlobalStreamThread; diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java index 560b8c89c60be..f270d3e14a3ec 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsBuilder.java @@ -487,7 +487,8 @@ public synchronized GlobalKTable globalTable(final String topic, /** * Adds a state store to the underlying {@link Topology}. *

- * It is required to connect state stores to {@link org.apache.kafka.streams.processor.Processor Processors}, {@link Transformer Transformers}, + * It is required to connect state stores to {@link org.apache.kafka.streams.processor.api.Processor Processors}, + * {@link Transformer Transformers}, * or {@link ValueTransformer ValueTransformers} before they can be used. * * @param builder the builder used to obtain this state store {@link StateStore} instance @@ -515,7 +516,8 @@ public synchronized StreamsBuilder addStateStore(final StoreBuilder builder) * This {@link ProcessorNode} should be used to keep the {@link StateStore} up-to-date. * The default {@link TimestampExtractor} as specified in the {@link StreamsConfig config} is used. *

- * It is not required to connect a global store to {@link org.apache.kafka.streams.processor.Processor Processors}, {@link Transformer Transformers}, + * It is not required to connect a global store to {@link org.apache.kafka.streams.processor.api.Processor Processors}, + * {@link Transformer Transformers}, * or {@link ValueTransformer ValueTransformer}; those have read-only access to all global stores by default. *

* The supplier should always generate a new instance each time {@link ProcessorSupplier#get()} gets called. Creating diff --git a/streams/src/main/java/org/apache/kafka/streams/Topology.java b/streams/src/main/java/org/apache/kafka/streams/Topology.java index f7cb8d679cb3a..7c45de7f47c9a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/Topology.java +++ b/streams/src/main/java/org/apache/kafka/streams/Topology.java @@ -22,11 +22,11 @@ import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.processor.ConnectedStoreProvider; -import org.apache.kafka.streams.processor.Processor; 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.Processor; import org.apache.kafka.streams.processor.api.ProcessorSupplier; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.processor.internals.ProcessorAdapter; @@ -655,7 +655,7 @@ public synchronized Topology addSink(final String name, * will be added to the topology and connected to this processor automatically. * * @param name the unique name of the processor node - * @param supplier the supplier used to obtain this node's {@link Processor} instance + * @param supplier the supplier used to obtain this node's {@link org.apache.kafka.streams.processor.Processor} instance * @param parentNames the name of one or more source or processor nodes whose output records this processor should receive * and process * @return itself diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java index 42bba28d13b43..f20d77ccb1c71 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java @@ -23,7 +23,7 @@ * This is a stateless record-by-record operation, i.e, {@link #apply(Object, Object)} is invoked individually for each * record of a stream. * If stateful processing is required, consider using - * {@link KStream#process(org.apache.kafka.streams.processor.ProcessorSupplier, String...) KStream#process(...)}. + * {@link KStream#process(org.apache.kafka.streams.processor.api.ProcessorSupplier, String...) KStream#process(...)}. * * @param key type * @param value type diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 00b9ee8991477..da7aa236b0493 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -25,9 +25,9 @@ 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.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.Processor; +import org.apache.kafka.streams.processor.api.ProcessorSupplier; +import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.TopicNameExtractor; import org.apache.kafka.streams.state.KeyValueStore; @@ -3173,12 +3173,13 @@ KStream leftJoin(final GlobalKTable globalTable, * (cf. {@link #transformValues(ValueTransformerSupplier, String...) transformValues()} ) *

* Note that it is possible to emit multiple records for each input record by using - * {@link ProcessorContext#forward(Object, Object) context#forward()} in + * {@link org.apache.kafka.streams.processor.ProcessorContext#forward(Object, Object) context#forward()} in * {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * Be aware that a mismatch between the types of the emitted records and the type of the stream would only be * detected at runtime. - * To ensure type-safety at compile-time, {@link ProcessorContext#forward(Object, Object) context#forward()} should + * To ensure type-safety at compile-time, + * {@link org.apache.kafka.streams.processor.ProcessorContext#forward(Object, Object) context#forward()} should * not be used in {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * If in {@link Transformer#transform(Object, Object) Transformer#transform()} multiple records need to be emitted @@ -3300,12 +3301,13 @@ KStream transform(final TransformerSupplier * Note that it is possible to emit multiple records for each input record by using - * {@link ProcessorContext#forward(Object, Object) context#forward()} in + * {@link org.apache.kafka.streams.processor.ProcessorContext#forward(Object, Object) context#forward()} in * {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * Be aware that a mismatch between the types of the emitted records and the type of the stream would only be * detected at runtime. - * To ensure type-safety at compile-time, {@link ProcessorContext#forward(Object, Object) context#forward()} should + * To ensure type-safety at compile-time, + * {@link org.apache.kafka.streams.processor.ProcessorContext#forward(Object, Object) context#forward()} should * not be used in {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * If in {@link Transformer#transform(Object, Object) Transformer#transform()} multiple records need to be emitted @@ -3432,12 +3434,14 @@ KStream transform(final TransformerSupplier - * Note that it is possible to emit records by using {@link ProcessorContext#forward(Object, Object) + * Note that it is possible to emit records by using + * {@link org.apache.kafka.streams.processor.ProcessorContext#forward(Object, Object) * context#forward()} in {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * Be aware that a mismatch between the types of the emitted records and the type of the stream would only be * detected at runtime. - * To ensure type-safety at compile-time, {@link ProcessorContext#forward(Object, Object) context#forward()} should + * To ensure type-safety at compile-time, + * {@link org.apache.kafka.streams.processor.ProcessorContext#forward(Object, Object) context#forward()} should * not be used in {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * The supplier should always generate a new instance each time {@link TransformerSupplier#get()} gets called. Creating @@ -3558,12 +3562,14 @@ KStream flatTransform(final TransformerSupplier - * Note that it is possible to emit records by using {@link ProcessorContext#forward(Object, Object) + * Note that it is possible to emit records by using + * {@link org.apache.kafka.streams.processor.ProcessorContext#forward(Object, Object) * context#forward()} in {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * Be aware that a mismatch between the types of the emitted records and the type of the stream would only be * detected at runtime. - * To ensure type-safety at compile-time, {@link ProcessorContext#forward(Object, Object) context#forward()} should + * To ensure type-safety at compile-time, + * {@link org.apache.kafka.streams.processor.ProcessorContext#forward(Object, Object) context#forward()} should * not be used in {@link Transformer#transform(Object, Object) Transformer#transform()} and * {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) Punctuator#punctuate()}. * The supplier should always generate a new instance each time {@link TransformerSupplier#get()} gets called. Creating @@ -3649,7 +3655,8 @@ KStream flatTransform(final TransformerSupplier{@code @@ -3757,7 +3764,8 @@ KStream transformValues(final ValueTransformerSupplier{@code @@ -3870,7 +3878,8 @@ KStream transformValues(final ValueTransformerSupplier{@code @@ -3982,7 +3991,8 @@ KStream transformValues(final ValueTransformerWithKeySupplier{@code @@ -4099,7 +4109,8 @@ KStream transformValues(final ValueTransformerWithKeySupplier{@code @@ -4221,7 +4232,8 @@ KStream flatTransformValues(final ValueTransformerSupplier{@code @@ -4345,7 +4357,8 @@ KStream flatTransformValues(final ValueTransformerSupplier{@code @@ -4468,7 +4481,8 @@ KStream flatTransformValues(final ValueTransformerWithKeySupplier{@code @@ -4524,6 +4538,109 @@ KStream flatTransformValues(final ValueTransformerWithKeySupplier + * 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). + *

+ * 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. + *

{@code
+     * // create store
+     * StoreBuilder> 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");
+     * }
+ * The second strategy is for the given {@link org.apache.kafka.streams.processor.ProcessorSupplier} + * to implement {@link ConnectedStoreProvider#stores()}, + * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the processor. + *
{@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 stores() {
+     *         StoreBuilder> keyValueStoreBuilder =
+     *                   Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("myProcessorState"),
+     *                   Serdes.String(),
+     *                   Serdes.String());
+     *         return Collections.singleton(keyValueStoreBuilder);
+     *     }
+     * }
+     *
+     * ...
+     *
+     * KStream outputStream = inputStream.process(new MyProcessorSupplier());
+     * }
+ *

+ * With either strategy, within the {@link org.apache.kafka.streams.processor.Processor}, + * the state is obtained via the {@link org.apache.kafka.streams.processor.ProcessorContext}. + * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, + * a schedule must be registered. + *

{@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
+     *     }
+     * }
+     * }
+ * 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 org.apache.kafka.streams.processor.ProcessorSupplier} + * that generates a newly constructed {@link org.apache.kafka.streams.processor.Processor} + * The supplier should always generate a new instance. Creating a single + * {@link org.apache.kafka.streams.processor.Processor} object + * and returning the same object reference in + * {@link org.apache.kafka.streams.processor.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...) + * @deprecated Since 3.0. Use {@link KStream#process(org.apache.kafka.streams.processor.api.ProcessorSupplier, java.lang.String...)} instead. + */ + @Deprecated + void process(final org.apache.kafka.streams.processor.ProcessorSupplier 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}). @@ -4615,7 +4732,110 @@ KStream flatTransformValues(final ValueTransformerWithKeySupplier processorSupplier, + void process(final ProcessorSupplier 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 + * {@link org.apache.kafka.streams.processor.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. + *

+ * 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). + *

+ * 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. + *

{@code
+     * // create store
+     * StoreBuilder> 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");
+     * }
+ * The second strategy is for the given {@link org.apache.kafka.streams.processor.ProcessorSupplier} + * to implement {@link ConnectedStoreProvider#stores()}, + * which provides the {@link StoreBuilder}s to be automatically added to the topology and connected to the processor. + *
{@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 stores() {
+     *         StoreBuilder> keyValueStoreBuilder =
+     *                   Stores.keyValueStoreBuilder(Stores.persistentKeyValueStore("myProcessorState"),
+     *                   Serdes.String(),
+     *                   Serdes.String());
+     *         return Collections.singleton(keyValueStoreBuilder);
+     *     }
+     * }
+     *
+     * ...
+     *
+     * KStream outputStream = inputStream.process(new MyProcessorSupplier());
+     * }
+ *

+ * With either strategy, within the {@link org.apache.kafka.streams.processor.Processor}, + * the state is obtained via the {@link org.apache.kafka.streams.processor.ProcessorContext}. + * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Punctuator#punctuate(long) punctuate()}, + * a schedule must be registered. + *

{@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
+     *     }
+     * }
+     * }
+ * 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 org.apache.kafka.streams.processor.ProcessorSupplier} + * that generates a newly constructed {@link org.apache.kafka.streams.processor.Processor} + * The supplier should always generate a new instance. Creating a single + * {@link org.apache.kafka.streams.processor.Processor} object + * and returning the same object reference in + * {@link org.apache.kafka.streams.processor.ProcessorSupplier#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 #foreach(ForeachAction) + * @see #transform(TransformerSupplier, String...) + * @deprecated Since 3.0. Use {@link KStream#process(org.apache.kafka.streams.processor.api.ProcessorSupplier, org.apache.kafka.streams.kstream.Named, java.lang.String...)} instead. + */ + @Deprecated + void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, + final Named named, final String... stateStoreNames); /** @@ -4709,7 +4929,7 @@ void process(final ProcessorSupplier processorSupplier, * @see #foreach(ForeachAction) * @see #transform(TransformerSupplier, String...) */ - void process(final ProcessorSupplier processorSupplier, + void process(final ProcessorSupplier processorSupplier, final Named named, final String... stateStoreNames); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java index c7585263a718a..80d9215460d61 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/CogroupedStreamAggregateBuilder.java @@ -38,7 +38,6 @@ import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters; import org.apache.kafka.streams.kstream.internals.graph.StatefulProcessorNode; import org.apache.kafka.streams.kstream.internals.graph.GraphNode; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.state.StoreBuilder; class CogroupedStreamAggregateBuilder { @@ -255,10 +254,11 @@ KTable createTable(final Collection processors, builder); } + @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. private StatefulProcessorNode getStatefulProcessorNode(final String processorName, final boolean stateCreated, final StoreBuilder storeBuilder, - final ProcessorSupplier kStreamAggregate) { + final org.apache.kafka.streams.processor.ProcessorSupplier kStreamAggregate) { final StatefulProcessorNode statefulProcessorNode; if (!stateCreated) { statefulProcessorNode = diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java index 4dde9c126c033..111d16968c4f8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java @@ -29,7 +29,6 @@ import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters; import org.apache.kafka.streams.kstream.internals.graph.StatefulProcessorNode; import org.apache.kafka.streams.kstream.internals.graph.GraphNode; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.state.KeyValueStore; import java.util.Collections; @@ -68,7 +67,8 @@ public class KGroupedTableImpl extends AbstractStream implements KGr this.userProvidedRepartitionTopicName = groupedInternal.name(); } - private KTable doAggregate(final ProcessorSupplier> aggregateSupplier, + @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. + private KTable doAggregate(final org.apache.kafka.streams.processor.ProcessorSupplier> aggregateSupplier, final NamedInternal named, final String functionName, final MaterializedInternal> materialized) { @@ -145,7 +145,8 @@ public KTable reduce(final Reducer adder, if (materializedInternal.valueSerde() == null) { materializedInternal.withValueSerde(valueSerde); } - final ProcessorSupplier> aggregateSupplier = new KTableReduce<>( + @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. + final org.apache.kafka.streams.processor.ProcessorSupplier> aggregateSupplier = new KTableReduce<>( materializedInternal.storeName(), adder, subtractor); @@ -176,7 +177,8 @@ public KTable count(final Named named, final Materialized> aggregateSupplier = new KTableAggregate<>( + @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. + final org.apache.kafka.streams.processor.ProcessorSupplier> aggregateSupplier = new KTableAggregate<>( materializedInternal.storeName(), countInitializer, countAdder, @@ -221,7 +223,8 @@ public KTable aggregate(final Initializer initializer, if (materializedInternal.keySerde() == null) { materializedInternal.withKeySerde(keySerde); } - final ProcessorSupplier> aggregateSupplier = new KTableAggregate<>( + @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. + final org.apache.kafka.streams.processor.ProcessorSupplier> aggregateSupplier = new KTableAggregate<>( materializedInternal.storeName(), initializer, adder, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggProcessorSupplier.java index 3f2186052371b..fab61be089ffb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggProcessorSupplier.java @@ -16,9 +16,8 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.processor.ProcessorSupplier; - -public interface KStreamAggProcessorSupplier extends ProcessorSupplier { +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. +public interface KStreamAggProcessorSupplier extends org.apache.kafka.streams.processor.ProcessorSupplier { KTableValueGetterSupplier view(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java index 4397fbc429d55..0f83849e37621 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java @@ -19,9 +19,6 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -31,6 +28,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. public class KStreamAggregate implements KStreamAggProcessorSupplier { private static final Logger LOG = LoggerFactory.getLogger(KStreamAggregate.class); private final String storeName; @@ -48,7 +46,7 @@ public class KStreamAggregate implements KStreamAggProcessorSupplier get() { + public org.apache.kafka.streams.processor.Processor get() { return new KStreamAggregateProcessor(); } @@ -58,13 +56,13 @@ public void enableSendingOldValues() { } - private class KStreamAggregateProcessor extends AbstractProcessor { + private class KStreamAggregateProcessor extends org.apache.kafka.streams.processor.AbstractProcessor { private TimestampedKeyValueStore store; private Sensor droppedRecordsSensor; private TimestampedTupleForwarder tupleForwarder; @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); droppedRecordsSensor = droppedRecordsSensor( Thread.currentThread().getName(), @@ -130,7 +128,7 @@ private class KStreamAggregateValueGetter implements KTableValueGetter { private TimestampedKeyValueStore store; @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { store = context.getStateStore(storeName); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java index 7836d440d372a..4d4fd2bf4740d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransform.java @@ -19,15 +19,12 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.Transformer; import org.apache.kafka.streams.kstream.TransformerSupplier; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.state.StoreBuilder; import java.util.Set; -public class KStreamFlatTransform implements ProcessorSupplier { +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. +public class KStreamFlatTransform implements org.apache.kafka.streams.processor.ProcessorSupplier { private final TransformerSupplier>> transformerSupplier; @@ -36,7 +33,7 @@ public KStreamFlatTransform(final TransformerSupplier get() { + public org.apache.kafka.streams.processor.Processor get() { return new KStreamFlatTransformProcessor<>(transformerSupplier.get()); } @@ -45,7 +42,7 @@ public Set> stores() { return transformerSupplier.stores(); } - public static class KStreamFlatTransformProcessor extends AbstractProcessor { + public static class KStreamFlatTransformProcessor extends org.apache.kafka.streams.processor.AbstractProcessor { private final Transformer>> transformer; @@ -54,7 +51,7 @@ public KStreamFlatTransformProcessor(final Transformer implements ProcessorSupplier { +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. +public class KStreamFlatTransformValues implements org.apache.kafka.streams.processor.ProcessorSupplier { private final ValueTransformerWithKeySupplier> valueTransformerSupplier; @@ -36,7 +33,7 @@ public KStreamFlatTransformValues(final ValueTransformerWithKeySupplier get() { + public org.apache.kafka.streams.processor.Processor get() { return new KStreamFlatTransformValuesProcessor<>(valueTransformerSupplier.get()); } @@ -45,7 +42,7 @@ public Set> stores() { return valueTransformerSupplier.stores(); } - public static class KStreamFlatTransformValuesProcessor extends AbstractProcessor { + public static class KStreamFlatTransformValuesProcessor extends org.apache.kafka.streams.processor.AbstractProcessor { private final ValueTransformerWithKey> valueTransformer; @@ -54,7 +51,7 @@ public static class KStreamFlatTransformValuesProcessor extends } @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); valueTransformer.init(new ForwardingDisabledProcessorContext(context)); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java index 568a639efeb53..be3fe6b343f47 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoin.java @@ -18,10 +18,9 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.ValueJoinerWithKey; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorSupplier; -class KStreamGlobalKTableJoin implements ProcessorSupplier { +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. +class KStreamGlobalKTableJoin implements org.apache.kafka.streams.processor.ProcessorSupplier { private final KTableValueGetterSupplier valueGetterSupplier; private final ValueJoinerWithKey joiner; @@ -39,7 +38,7 @@ class KStreamGlobalKTableJoin implements ProcessorSupplier get() { + public org.apache.kafka.streams.processor.Processor get() { return new KStreamKTableJoinProcessor<>(valueGetterSupplier.get(), mapper, joiner, leftJoin); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 7ac688108970d..f7075f6bb139c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -58,7 +58,7 @@ 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.ProcessorSupplier; +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; @@ -1215,7 +1215,9 @@ private KStream globalTableJoin(final GlobalKTable g final KTableValueGetterSupplier valueGetterSupplier = ((GlobalKTableImpl) globalTable).valueGetterSupplier(); final String name = new NamedInternal(named).orElseGenerateWithPrefix(builder, LEFTJOIN_NAME); - final ProcessorSupplier processorSupplier = new KStreamGlobalKTableJoin<>( + // Old PAPI. Needs to be migrated. + @SuppressWarnings("deprecation") + final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier = new KStreamGlobalKTableJoin<>( valueGetterSupplier, joiner, keySelector, @@ -1251,7 +1253,9 @@ private KStream doStreamTableJoin(final KTable table, final NamedInternal renamed = new NamedInternal(joinedInternal.name()); final String name = renamed.orElseGenerateWithPrefix(builder, leftJoin ? LEFTJOIN_NAME : JOIN_NAME); - final ProcessorSupplier processorSupplier = new KStreamKTableJoin<>( + // Old PAPI. Needs to be migrated. + @SuppressWarnings("deprecation") + final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier = new KStreamKTableJoin<>( ((KTableImpl) table).valueGetterSupplier(), joiner, leftJoin); @@ -1466,13 +1470,42 @@ private KStream doFlatTransformValues(final ValueTransformerWithKeyS } @Override - public void process(final ProcessorSupplier processorSupplier, + @Deprecated + public void process(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, final String... stateStoreNames) { process(processorSupplier, Named.as(builder.newProcessorName(PROCESSOR_NAME)), stateStoreNames); } @Override - public void process(final ProcessorSupplier processorSupplier, + public void process(final ProcessorSupplier 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 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 processNode = new StatefulProcessorNode<>( + name, + new ProcessorParameters<>(processorSupplier, name), + stateStoreNames); + + builder.addGraphNode(graphNode, processNode); + } + + @Override + public void process(final ProcessorSupplier processorSupplier, final Named named, final String... stateStoreNames) { Objects.requireNonNull(processorSupplier, "processorSupplier can't be null"); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java index 34756d47cf2ee..87edbbc2c2748 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java @@ -16,13 +16,10 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.state.WindowStore; -class KStreamJoinWindow implements ProcessorSupplier { +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. +class KStreamJoinWindow implements org.apache.kafka.streams.processor.ProcessorSupplier { private final String windowName; @@ -31,17 +28,17 @@ class KStreamJoinWindow implements ProcessorSupplier { } @Override - public Processor get() { + public org.apache.kafka.streams.processor.Processor get() { return new KStreamJoinWindowProcessor(); } - private class KStreamJoinWindowProcessor extends AbstractProcessor { + private class KStreamJoinWindowProcessor extends org.apache.kafka.streams.processor.AbstractProcessor { private WindowStore window; @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); window = (WindowStore) context.getStateStore(windowName); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index 52853fbbdaa41..fcf1cf944944d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -21,10 +21,6 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.ValueJoinerWithKey; import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueIterator; @@ -40,7 +36,8 @@ import static org.apache.kafka.streams.StreamsConfig.InternalConfig.ENABLE_KSTREAMS_OUTER_JOIN_SPURIOUS_RESULTS_FIX; import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor; -class KStreamKStreamJoin implements ProcessorSupplier { +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. +class KStreamKStreamJoin implements org.apache.kafka.streams.processor.ProcessorSupplier { private static final Logger LOG = LoggerFactory.getLogger(KStreamKStreamJoin.class); private final String otherWindowName; @@ -76,17 +73,17 @@ class KStreamKStreamJoin implements ProcessorSupplier { } @Override - public Processor get() { + public org.apache.kafka.streams.processor.Processor get() { return new KStreamKStreamJoinProcessor(); } - private class KStreamKStreamJoinProcessor extends AbstractProcessor { + private class KStreamKStreamJoinProcessor extends org.apache.kafka.streams.processor.AbstractProcessor { private WindowStore otherWindowStore; private Sensor droppedRecordsSensor; private Optional, LeftOrRightValue>> outerJoinWindowStore = Optional.empty(); @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); final StreamsMetricsImpl metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java index 136b953441b29..deca86d21b3ec 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoin.java @@ -18,10 +18,9 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.ValueJoinerWithKey; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorSupplier; -class KStreamKTableJoin implements ProcessorSupplier { +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. +class KStreamKTableJoin implements org.apache.kafka.streams.processor.ProcessorSupplier { private final KeyValueMapper keyValueMapper = (key, value) -> key; private final KTableValueGetterSupplier valueGetterSupplier; @@ -37,7 +36,7 @@ class KStreamKTableJoin implements ProcessorSupplier { } @Override - public Processor get() { + public org.apache.kafka.streams.processor.Processor get() { return new KStreamKTableJoinProcessor<>(valueGetterSupplier.get(), keyValueMapper, joiner, leftJoin); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java index 29c49e8915cff..38e839c130795 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinProcessor.java @@ -19,8 +19,6 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.ValueJoinerWithKey; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -28,7 +26,8 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; -class KStreamKTableJoinProcessor extends AbstractProcessor { +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. +class KStreamKTableJoinProcessor extends org.apache.kafka.streams.processor.AbstractProcessor { private static final Logger LOG = LoggerFactory.getLogger(KStreamKTableJoin.class); private final KTableValueGetter valueGetter; @@ -48,7 +47,7 @@ class KStreamKTableJoinProcessor extends AbstractProcessor implements KStreamAggProcessorSupplier { private static final Logger LOG = LoggerFactory.getLogger(KStreamReduce.class); @@ -44,7 +42,7 @@ public class KStreamReduce implements KStreamAggProcessorSupplier get() { + public org.apache.kafka.streams.processor.Processor get() { return new KStreamReduceProcessor(); } @@ -54,13 +52,13 @@ public void enableSendingOldValues() { } - private class KStreamReduceProcessor extends AbstractProcessor { + private class KStreamReduceProcessor extends org.apache.kafka.streams.processor.AbstractProcessor { private TimestampedKeyValueStore store; private TimestampedTupleForwarder tupleForwarder; private Sensor droppedRecordsSensor; @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); droppedRecordsSensor = droppedRecordsSensor( Thread.currentThread().getName(), @@ -126,7 +124,7 @@ private class KStreamReduceValueGetter implements KTableValueGetter { private TimestampedKeyValueStore store; @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { store = context.getStateStore(storeName); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java index 413d6ca1098f5..a2fce1edf4ac1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregate.java @@ -24,9 +24,6 @@ import org.apache.kafka.streams.kstream.Merger; import org.apache.kafka.streams.kstream.SessionWindows; import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.SessionStore; @@ -39,6 +36,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor; +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. public class KStreamSessionWindowAggregate implements KStreamAggProcessorSupplier, V, Agg> { private static final Logger LOG = LoggerFactory.getLogger(KStreamSessionWindowAggregate.class); @@ -63,7 +61,7 @@ public KStreamSessionWindowAggregate(final SessionWindows windows, } @Override - public Processor get() { + public org.apache.kafka.streams.processor.Processor get() { return new KStreamSessionWindowAggregateProcessor(); } @@ -76,7 +74,7 @@ public void enableSendingOldValues() { sendOldValues = true; } - private class KStreamSessionWindowAggregateProcessor extends AbstractProcessor { + private class KStreamSessionWindowAggregateProcessor extends org.apache.kafka.streams.processor.AbstractProcessor { private SessionStore store; private SessionTupleForwarder tupleForwarder; @@ -85,7 +83,7 @@ private class KStreamSessionWindowAggregateProcessor extends AbstractProcessor store; @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { store = context.getStateStore(storeName); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java index c3ae7ceae0f48..64b747a9ab342 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java @@ -24,9 +24,6 @@ import org.apache.kafka.streams.kstream.Window; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.SlidingWindows; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.KeyValueIterator; @@ -40,6 +37,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. public class KStreamSlidingWindowAggregate implements KStreamAggProcessorSupplier, V, Agg> { private final Logger log = LoggerFactory.getLogger(getClass()); @@ -61,7 +59,7 @@ public KStreamSlidingWindowAggregate(final SlidingWindows windows, } @Override - public Processor get() { + public org.apache.kafka.streams.processor.Processor get() { return new KStreamSlidingWindowAggregateProcessor(); } @@ -74,7 +72,7 @@ public void enableSendingOldValues() { sendOldValues = true; } - private class KStreamSlidingWindowAggregateProcessor extends AbstractProcessor { + private class KStreamSlidingWindowAggregateProcessor extends org.apache.kafka.streams.processor.AbstractProcessor { private TimestampedWindowStore windowStore; private TimestampedTupleForwarder, Agg> tupleForwarder; private Sensor lateRecordDropSensor; @@ -83,7 +81,7 @@ private class KStreamSlidingWindowAggregateProcessor extends AbstractProcessor windowStore; @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { windowStore = context.getStateStore(storeName); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java index e2be315e0c142..468bf8dfa2b8b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java @@ -18,16 +18,13 @@ import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; import org.apache.kafka.streams.state.StoreBuilder; import java.util.Set; -public class KStreamTransformValues implements ProcessorSupplier { +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. +public class KStreamTransformValues implements org.apache.kafka.streams.processor.ProcessorSupplier { private final ValueTransformerWithKeySupplier valueTransformerSupplier; @@ -36,7 +33,7 @@ public class KStreamTransformValues implements ProcessorSupplier } @Override - public Processor get() { + public org.apache.kafka.streams.processor.Processor get() { return new KStreamTransformValuesProcessor<>(valueTransformerSupplier.get()); } @@ -45,7 +42,7 @@ public Set> stores() { return valueTransformerSupplier.stores(); } - public static class KStreamTransformValuesProcessor extends AbstractProcessor { + public static class KStreamTransformValuesProcessor extends org.apache.kafka.streams.processor.AbstractProcessor { private final ValueTransformerWithKey valueTransformer; @@ -54,7 +51,7 @@ public static class KStreamTransformValuesProcessor extends AbstractPro } @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); valueTransformer.init(new ForwardingDisabledProcessorContext(context)); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java index 300f3872e4d3f..174158dc77a11 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java @@ -23,9 +23,6 @@ import org.apache.kafka.streams.kstream.Window; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.Windows; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.TimestampedWindowStore; @@ -38,6 +35,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. public class KStreamWindowAggregate implements KStreamAggProcessorSupplier, V, Agg> { private final Logger log = LoggerFactory.getLogger(getClass()); @@ -59,7 +57,7 @@ public KStreamWindowAggregate(final Windows windows, } @Override - public Processor get() { + public org.apache.kafka.streams.processor.Processor get() { return new KStreamWindowAggregateProcessor(); } @@ -73,7 +71,7 @@ public void enableSendingOldValues() { } - private class KStreamWindowAggregateProcessor extends AbstractProcessor { + private class KStreamWindowAggregateProcessor extends org.apache.kafka.streams.processor.AbstractProcessor { private TimestampedWindowStore windowStore; private TimestampedTupleForwarder, Agg> tupleForwarder; private Sensor lateRecordDropSensor; @@ -81,7 +79,7 @@ private class KStreamWindowAggregateProcessor extends AbstractProcessor { private long observedStreamTime = ConsumerRecord.NO_TIMESTAMP; @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); final InternalProcessorContext internalProcessorContext = (InternalProcessorContext) context; final StreamsMetricsImpl metrics = internalProcessorContext.metrics(); @@ -194,7 +192,7 @@ private class KStreamWindowAggregateValueGetter implements KTableValueGetter windowStore; @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { windowStore = context.getStateStore(storeName); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java index 1e6474627d6ff..3ff4dfa78fd0d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java @@ -19,14 +19,12 @@ import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. public class KTableAggregate implements KTableProcessorSupplier { private final String storeName; @@ -54,17 +52,17 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { } @Override - public Processor> get() { + public org.apache.kafka.streams.processor.Processor> get() { return new KTableAggregateProcessor(); } - private class KTableAggregateProcessor extends AbstractProcessor> { + private class KTableAggregateProcessor extends org.apache.kafka.streams.processor.AbstractProcessor> { private TimestampedKeyValueStore store; private TimestampedTupleForwarder tupleForwarder; @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); store = (TimestampedKeyValueStore) context.getStateStore(storeName); tupleForwarder = new TimestampedTupleForwarder<>( diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index faee7af4ce401..5eaf2a6daa958 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -58,7 +58,6 @@ import org.apache.kafka.streams.kstream.internals.suppress.KTableSuppressProcessorSupplier; import org.apache.kafka.streams.kstream.internals.suppress.NamedSuppressed; import org.apache.kafka.streams.kstream.internals.suppress.SuppressedInternal; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.internals.InternalTopicProperties; import org.apache.kafka.streams.processor.internals.StaticTopicNameExtractor; import org.apache.kafka.streams.state.KeyValueStore; @@ -133,12 +132,13 @@ public class KTableImpl extends AbstractStream implements KTable< private boolean sendOldValues = false; + @SuppressWarnings("deprecation") // Old PAPI compatibility. public KTableImpl(final String name, final Serde keySerde, final Serde valueSerde, final Set subTopologySourceNodes, final String queryableStoreName, - final ProcessorSupplier processorSupplier, + final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, final GraphNode graphNode, final InternalStreamsBuilder builder) { super(name, keySerde, valueSerde, subTopologySourceNodes, graphNode, builder); @@ -542,7 +542,8 @@ public KTable suppress(final Suppressed suppressed) { final String storeName = suppressedInternal.name() != null ? suppressedInternal.name() + "-store" : builder.newStoreName(SUPPRESS_NAME); - final ProcessorSupplier> suppressionSupplier = new KTableSuppressProcessorSupplier<>( + @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. + final org.apache.kafka.streams.processor.ProcessorSupplier> suppressionSupplier = new KTableSuppressProcessorSupplier<>( suppressedInternal, storeName, this diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java index 4626c365dea6c..e448aefaf159b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoin.java @@ -19,9 +19,6 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -31,6 +28,7 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. class KTableKTableInnerJoin extends KTableKTableAbstractJoin { private static final Logger LOG = LoggerFactory.getLogger(KTableKTableInnerJoin.class); @@ -43,7 +41,7 @@ class KTableKTableInnerJoin extends KTableKTableAbstractJoin> get() { + public org.apache.kafka.streams.processor.Processor> get() { return new KTableKTableJoinProcessor(valueGetterSupplier2.get()); } @@ -64,7 +62,7 @@ public KTableValueGetter get() { } } - private class KTableKTableJoinProcessor extends AbstractProcessor> { + private class KTableKTableJoinProcessor extends org.apache.kafka.streams.processor.AbstractProcessor> { private final KTableValueGetter valueGetter; private Sensor droppedRecordsSensor; @@ -74,7 +72,7 @@ private class KTableKTableJoinProcessor extends AbstractProcessor> } @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); droppedRecordsSensor = droppedRecordsSensor( Thread.currentThread().getName(), @@ -137,7 +135,7 @@ private class KTableKTableInnerJoinValueGetter implements KTableValueGetter implements KTableProcessorSupplier { private final KTableProcessorSupplier parent1; @@ -46,7 +44,7 @@ public String getQueryableName() { } @Override - public Processor> get() { + public org.apache.kafka.streams.processor.Processor> get() { return new KTableKTableJoinMergeProcessor(); } @@ -96,13 +94,13 @@ public static KTableKTableJoinMerger of(final KTableProcessorSuppli return new KTableKTableJoinMerger<>(parent1, parent2, queryableName); } - private class KTableKTableJoinMergeProcessor extends AbstractProcessor> { + private class KTableKTableJoinMergeProcessor extends org.apache.kafka.streams.processor.AbstractProcessor> { private TimestampedKeyValueStore store; private TimestampedTupleForwarder tupleForwarder; @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); if (queryableName != null) { store = (TimestampedKeyValueStore) context.getStateStore(queryableName); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index abc28c0a477b3..dc274cdae0223 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -18,9 +18,6 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -31,6 +28,7 @@ import static org.apache.kafka.streams.processor.internals.RecordQueue.UNKNOWN; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. class KTableKTableLeftJoin extends KTableKTableAbstractJoin { private static final Logger LOG = LoggerFactory.getLogger(KTableKTableLeftJoin.class); @@ -41,7 +39,7 @@ class KTableKTableLeftJoin extends KTableKTableAbstractJoin> get() { + public org.apache.kafka.streams.processor.Processor> get() { return new KTableKTableLeftJoinProcessor(valueGetterSupplier2.get()); } @@ -63,7 +61,7 @@ public KTableValueGetter get() { } - private class KTableKTableLeftJoinProcessor extends AbstractProcessor> { + private class KTableKTableLeftJoinProcessor extends org.apache.kafka.streams.processor.AbstractProcessor> { private final KTableValueGetter valueGetter; private Sensor droppedRecordsSensor; @@ -73,7 +71,7 @@ private class KTableKTableLeftJoinProcessor extends AbstractProcessor } @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { valueGetter1.init(context); valueGetter2.init(context); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java index 4eaee6b475820..6b2017a439655 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java @@ -18,9 +18,6 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -31,6 +28,7 @@ import static org.apache.kafka.streams.processor.internals.RecordQueue.UNKNOWN; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. class KTableKTableOuterJoin extends KTableKTableAbstractJoin { private static final Logger LOG = LoggerFactory.getLogger(KTableKTableOuterJoin.class); @@ -41,7 +39,7 @@ class KTableKTableOuterJoin extends KTableKTableAbstractJoin> get() { + public org.apache.kafka.streams.processor.Processor> get() { return new KTableKTableOuterJoinProcessor(valueGetterSupplier2.get()); } @@ -62,7 +60,7 @@ public KTableValueGetter get() { } } - private class KTableKTableOuterJoinProcessor extends AbstractProcessor> { + private class KTableKTableOuterJoinProcessor extends org.apache.kafka.streams.processor.AbstractProcessor> { private final KTableValueGetter valueGetter; private Sensor droppedRecordsSensor; @@ -72,7 +70,7 @@ private class KTableKTableOuterJoinProcessor extends AbstractProcessor extends KTableKTableAbstractJoin { private static final Logger LOG = LoggerFactory.getLogger(KTableKTableRightJoin.class); @@ -40,7 +38,7 @@ class KTableKTableRightJoin extends KTableKTableAbstractJoin> get() { + public org.apache.kafka.streams.processor.Processor> get() { return new KTableKTableRightJoinProcessor(valueGetterSupplier2.get()); } @@ -61,7 +59,7 @@ public KTableValueGetter get() { } } - private class KTableKTableRightJoinProcessor extends AbstractProcessor> { + private class KTableKTableRightJoinProcessor extends org.apache.kafka.streams.processor.AbstractProcessor> { private final KTableValueGetter valueGetter; private Sensor droppedRecordsSensor; @@ -71,7 +69,7 @@ private class KTableKTableRightJoinProcessor extends AbstractProcessor implements KTableProcessorSupplier { private final KTableImpl parent; private final ValueMapperWithKey mapper; @@ -41,7 +39,7 @@ class KTableMapValues implements KTableProcessorSupplier { } @Override - public Processor> get() { + public org.apache.kafka.streams.processor.Processor> get() { return new KTableMapValuesProcessor(); } @@ -104,12 +102,12 @@ private ValueAndTimestamp computeValueAndTimestamp(final K key, final ValueA } - private class KTableMapValuesProcessor extends AbstractProcessor> { + private class KTableMapValuesProcessor extends org.apache.kafka.streams.processor.AbstractProcessor> { private TimestampedKeyValueStore store; private TimestampedTupleForwarder tupleForwarder; @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); if (queryableName != null) { store = context.getStateStore(queryableName); @@ -154,7 +152,7 @@ private class KTableMapValuesValueGetter implements KTableValueGetter { } @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { parentGetter.init(context); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java index 2f16d0e3c064d..8a4947969bdcb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTablePassThrough.java @@ -16,14 +16,12 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; import java.util.Collection; +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. public class KTablePassThrough implements KTableProcessorSupplier { private final Collection parents; private final String storeName; @@ -35,7 +33,7 @@ public class KTablePassThrough implements KTableProcessorSupplier } @Override - public Processor> get() { + public org.apache.kafka.streams.processor.Processor> get() { return new KTablePassThroughProcessor(); } @@ -64,7 +62,7 @@ public String[] storeNames() { }; } - private class KTablePassThroughProcessor extends AbstractProcessor> { + private class KTablePassThroughProcessor extends org.apache.kafka.streams.processor.AbstractProcessor> { @Override public void process(final K key, final Change value) { context().forward(key, value); @@ -75,7 +73,7 @@ private class KTablePassThroughValueGetter implements KTableValueGetter { private TimestampedKeyValueStore store; @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { store = context.getStateStore(storeName); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableProcessorSupplier.java index ff6f9d61df969..6f30dbb476340 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableProcessorSupplier.java @@ -16,9 +16,8 @@ */ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.processor.ProcessorSupplier; - -public interface KTableProcessorSupplier extends ProcessorSupplier> { +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. +public interface KTableProcessorSupplier extends org.apache.kafka.streams.processor.ProcessorSupplier> { KTableValueGetterSupplier view(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java index 88f62f96d5d6b..72f75ea25ded4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java @@ -18,14 +18,12 @@ import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.Reducer; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. public class KTableReduce implements KTableProcessorSupplier { private final String storeName; @@ -48,18 +46,18 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { } @Override - public Processor> get() { + public org.apache.kafka.streams.processor.Processor> get() { return new KTableReduceProcessor(); } - private class KTableReduceProcessor extends AbstractProcessor> { + private class KTableReduceProcessor extends org.apache.kafka.streams.processor.AbstractProcessor> { private TimestampedKeyValueStore store; private TimestampedTupleForwarder tupleForwarder; @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); store = (TimestampedKeyValueStore) context.getStateStore(storeName); tupleForwarder = new TimestampedTupleForwarder<>( diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java index d5dc5db0f44d8..6df6ce5ff95e6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java @@ -19,9 +19,6 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.state.ValueAndTimestamp; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; @@ -31,6 +28,7 @@ *

* Given the input, it can output at most two records (one mapped from old value and one mapped from new value). */ +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. public class KTableRepartitionMap implements KTableProcessorSupplier> { private final KTableImpl parent; @@ -42,7 +40,7 @@ public class KTableRepartitionMap implements KTableProcessorSuppli } @Override - public Processor> get() { + public org.apache.kafka.streams.processor.Processor> get() { return new KTableMapProcessor(); } @@ -72,7 +70,7 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { throw new IllegalStateException("KTableRepartitionMap should always require sending old values."); } - private class KTableMapProcessor extends AbstractProcessor> { + private class KTableMapProcessor extends org.apache.kafka.streams.processor.AbstractProcessor> { /** * @throws StreamsException if key is null @@ -103,14 +101,14 @@ public void process(final K key, final Change change) { private class KTableMapValueGetter implements KTableValueGetter> { private final KTableValueGetter parentGetter; - private ProcessorContext context; + private org.apache.kafka.streams.processor.ProcessorContext context; KTableMapValueGetter(final KTableValueGetter parentGetter) { this.parentGetter = parentGetter; } @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { this.context = context; parentGetter.init(context); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java index 1e8835d8d9897..04dbd95498a4c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java @@ -17,10 +17,6 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.TimestampedKeyValueStore; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -31,7 +27,8 @@ import static org.apache.kafka.streams.processor.internals.metrics.TaskMetrics.droppedRecordsSensor; -public class KTableSource implements ProcessorSupplier { +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. +public class KTableSource implements org.apache.kafka.streams.processor.ProcessorSupplier { private static final Logger LOG = LoggerFactory.getLogger(KTableSource.class); private final String storeName; @@ -51,7 +48,7 @@ public String queryableName() { } @Override - public Processor get() { + public org.apache.kafka.streams.processor.Processor get() { return new KTableSourceProcessor(); } @@ -72,7 +69,7 @@ public boolean materialized() { return queryableName != null; } - private class KTableSourceProcessor extends AbstractProcessor { + private class KTableSourceProcessor extends org.apache.kafka.streams.processor.AbstractProcessor { private TimestampedKeyValueStore store; private TimestampedTupleForwarder tupleForwarder; @@ -80,7 +77,7 @@ private class KTableSourceProcessor extends AbstractProcessor { @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); final StreamsMetricsImpl metrics = (StreamsMetricsImpl) context.metrics(); droppedRecordsSensor = droppedRecordsSensor(Thread.currentThread().getName(), context.taskId().toString(), metrics); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java index f45d32c613da6..94a1c0e629290 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableTransformValues.java @@ -19,9 +19,6 @@ import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.streams.kstream.ValueTransformerWithKey; import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; @@ -33,6 +30,7 @@ import static org.apache.kafka.streams.processor.internals.RecordQueue.UNKNOWN; import static org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull; +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. class KTableTransformValues implements KTableProcessorSupplier { private final KTableImpl parent; private final ValueTransformerWithKeySupplier transformerSupplier; @@ -48,7 +46,7 @@ class KTableTransformValues implements KTableProcessorSupplier> get() { + public org.apache.kafka.streams.processor.Processor> get() { return new KTableTransformValuesProcessor(transformerSupplier.get()); } @@ -87,7 +85,7 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { return sendOldValues; } - private class KTableTransformValuesProcessor extends AbstractProcessor> { + private class KTableTransformValuesProcessor extends org.apache.kafka.streams.processor.AbstractProcessor> { private final ValueTransformerWithKey valueTransformer; private TimestampedKeyValueStore store; private TimestampedTupleForwarder tupleForwarder; @@ -97,7 +95,7 @@ private KTableTransformValuesProcessor(final ValueTransformerWithKey { } @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { internalProcessorContext = (InternalProcessorContext) context; parentGetter.init(context); valueTransformer.init(new ForwardingDisabledProcessorContext(context)); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java index 086c83022add6..7b1ec66bf6dfb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionProcessorSupplier.java @@ -22,10 +22,6 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.internals.Change; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.state.KeyValueIterator; @@ -37,7 +33,8 @@ import java.nio.ByteBuffer; -public class ForeignJoinSubscriptionProcessorSupplier implements ProcessorSupplier> { +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. +public class ForeignJoinSubscriptionProcessorSupplier implements org.apache.kafka.streams.processor.ProcessorSupplier> { private static final Logger LOG = LoggerFactory.getLogger(ForeignJoinSubscriptionProcessorSupplier.class); private final StoreBuilder>> storeBuilder; private final CombinedKeySchema keySchema; @@ -51,17 +48,17 @@ public ForeignJoinSubscriptionProcessorSupplier( } @Override - public Processor> get() { + public org.apache.kafka.streams.processor.Processor> get() { return new KTableKTableJoinProcessor(); } - private final class KTableKTableJoinProcessor extends AbstractProcessor> { + private final class KTableKTableJoinProcessor extends org.apache.kafka.streams.processor.AbstractProcessor> { private Sensor droppedRecordsSensor; private TimestampedKeyValueStore> store; @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); final InternalProcessorContext internalProcessorContext = (InternalProcessorContext) context; droppedRecordsSensor = TaskMetrics.droppedRecordsSensor( diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java index 4e19b4f64b2aa..8f1e1f9e65495 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/ForeignJoinSubscriptionSendProcessorSupplier.java @@ -21,10 +21,6 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.kstream.internals.Change; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; import org.apache.kafka.streams.state.internals.Murmur3; @@ -40,7 +36,8 @@ import static org.apache.kafka.streams.kstream.internals.foreignkeyjoin.SubscriptionWrapper.Instruction.PROPAGATE_NULL_IF_NO_FK_VAL_AVAILABLE; import static org.apache.kafka.streams.kstream.internals.foreignkeyjoin.SubscriptionWrapper.Instruction.PROPAGATE_ONLY_IF_FK_VAL_AVAILABLE; -public class ForeignJoinSubscriptionSendProcessorSupplier implements ProcessorSupplier> { +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. +public class ForeignJoinSubscriptionSendProcessorSupplier implements org.apache.kafka.streams.processor.ProcessorSupplier> { private static final Logger LOG = LoggerFactory.getLogger(ForeignJoinSubscriptionSendProcessorSupplier.class); private final Function foreignKeyExtractor; @@ -65,11 +62,11 @@ public ForeignJoinSubscriptionSendProcessorSupplier(final Function foreig } @Override - public Processor> get() { + public org.apache.kafka.streams.processor.Processor> get() { return new UnbindChangeProcessor(); } - private class UnbindChangeProcessor extends AbstractProcessor> { + private class UnbindChangeProcessor extends org.apache.kafka.streams.processor.AbstractProcessor> { private Sensor droppedRecordsSensor; private String foreignKeySerdeTopic; @@ -77,7 +74,7 @@ private class UnbindChangeProcessor extends AbstractProcessor> { @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); foreignKeySerdeTopic = foreignKeySerdeTopicSupplier.get(); valueSerdeTopic = valueSerdeTopicSupplier.get(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java index 2544eb1856b7c..7c53a6881f21f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionJoinForeignProcessorSupplier.java @@ -21,10 +21,6 @@ import org.apache.kafka.streams.kstream.internals.Change; import org.apache.kafka.streams.kstream.internals.KTableValueGetter; import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.state.ValueAndTimestamp; @@ -39,8 +35,9 @@ * @param Type of foreign key * @param Type of foreign value */ +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. public class SubscriptionJoinForeignProcessorSupplier - implements ProcessorSupplier, Change>>> { + implements org.apache.kafka.streams.processor.ProcessorSupplier, Change>>> { private final KTableValueGetterSupplier foreignValueGetterSupplier; @@ -49,14 +46,14 @@ public SubscriptionJoinForeignProcessorSupplier(final KTableValueGetterSupplier< } @Override - public Processor, Change>>> get() { + public org.apache.kafka.streams.processor.Processor, Change>>> get() { - return new AbstractProcessor, Change>>>() { + return new org.apache.kafka.streams.processor.AbstractProcessor, Change>>>() { private KTableValueGetter foreignValues; @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); foreignValues = foreignValueGetterSupplier.get(); foreignValues.init(context); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java index 3cd06368a7131..652adbd102983 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplier.java @@ -22,10 +22,6 @@ import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.internals.KTableValueGetter; import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.internals.Murmur3; @@ -41,7 +37,8 @@ * @param Type of foreign values * @param Type of joined result of primary and foreign values */ -public class SubscriptionResolverJoinProcessorSupplier implements ProcessorSupplier> { +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. +public class SubscriptionResolverJoinProcessorSupplier implements org.apache.kafka.streams.processor.ProcessorSupplier> { private final KTableValueGetterSupplier valueGetterSupplier; private final Serializer constructionTimeValueSerializer; private final Supplier valueHashSerdePseudoTopicSupplier; @@ -61,8 +58,8 @@ public SubscriptionResolverJoinProcessorSupplier(final KTableValueGetterSupplier } @Override - public Processor> get() { - return new AbstractProcessor>() { + public org.apache.kafka.streams.processor.Processor> get() { + return new org.apache.kafka.streams.processor.AbstractProcessor>() { private String valueHashSerdePseudoTopic; private Serializer runtimeValueSerializer = constructionTimeValueSerializer; @@ -70,7 +67,7 @@ public Processor> get() { @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); valueHashSerdePseudoTopic = valueHashSerdePseudoTopicSupplier.get(); valueGetter = valueGetterSupplier.get(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java index 4746fc6e9461f..928bd4831a8cd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionStoreReceiveProcessorSupplier.java @@ -21,10 +21,6 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.kstream.internals.Change; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.To; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics; @@ -34,8 +30,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. public class SubscriptionStoreReceiveProcessorSupplier - implements ProcessorSupplier> { + implements org.apache.kafka.streams.processor.ProcessorSupplier> { private static final Logger LOG = LoggerFactory.getLogger(SubscriptionStoreReceiveProcessorSupplier.class); private final StoreBuilder>> storeBuilder; @@ -50,15 +47,15 @@ public SubscriptionStoreReceiveProcessorSupplier( } @Override - public Processor> get() { + public org.apache.kafka.streams.processor.Processor> get() { - return new AbstractProcessor>() { + return new org.apache.kafka.streams.processor.AbstractProcessor>() { private TimestampedKeyValueStore> store; private Sensor droppedRecordsSensor; @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); final InternalProcessorContext internalProcessorContext = (InternalProcessorContext) context; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtil.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtil.java index 017087e6404c8..4b39219a371e5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtil.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtil.java @@ -23,7 +23,6 @@ import org.apache.kafka.streams.kstream.internals.KStreamSessionWindowAggregate; import org.apache.kafka.streams.kstream.internals.KStreamSlidingWindowAggregate; import org.apache.kafka.streams.kstream.internals.KStreamWindowAggregate; -import org.apache.kafka.streams.processor.ProcessorSupplier; public final class GraphGraceSearchUtil { private GraphGraceSearchUtil() {} @@ -72,7 +71,8 @@ private static long findAndVerifyWindowGrace(final GraphNode graphNode, final St private static Long extractGracePeriod(final GraphNode node) { if (node instanceof StatefulProcessorNode) { - final ProcessorSupplier processorSupplier = ((StatefulProcessorNode) node).processorParameters().oldProcessorSupplier(); + @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. + final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier = ((StatefulProcessorNode) node).processorParameters().oldProcessorSupplier(); if (processorSupplier instanceof KStreamWindowAggregate) { final KStreamWindowAggregate kStreamWindowAggregate = (KStreamWindowAggregate) processorSupplier; final Windows windows = kStreamWindowAggregate.windows(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java index 018d2b7dc7dd3..9e23c5d418b61 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java @@ -34,10 +34,12 @@ public class ProcessorParameters { // During the transition to KIP-478, we capture arguments passed from the old API to simplify // the performance of casts that we still need to perform. This will eventually be removed. + @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. private final org.apache.kafka.streams.processor.ProcessorSupplier oldProcessorSupplier; private final ProcessorSupplier processorSupplier; private final String processorName; + @SuppressWarnings("deprecation") // Old PAPI compatibility. public ProcessorParameters(final org.apache.kafka.streams.processor.ProcessorSupplier processorSupplier, final String processorName) { oldProcessorSupplier = processorSupplier; @@ -56,6 +58,7 @@ public ProcessorSupplier processorSupplier() { return processorSupplier; } + @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. public org.apache.kafka.streams.processor.ProcessorSupplier oldProcessorSupplier() { return oldProcessorSupplier; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java index ad6de37f45cbe..88264f91e68f6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StatefulProcessorNode.java @@ -101,6 +101,7 @@ public void writeToTopology(final InternalTopologyBuilder topologyBuilder, final } // temporary hack until KIP-478 is fully implemented + @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. final org.apache.kafka.streams.processor.ProcessorSupplier oldProcessorSupplier = processorParameters().oldProcessorSupplier(); if (oldProcessorSupplier != null && oldProcessorSupplier.stores() != null) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java index 495f6d04ecac5..5a9f4822aec99 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorSupplier.java @@ -26,9 +26,6 @@ import org.apache.kafka.streams.kstream.internals.KTableValueGetter; import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier; import org.apache.kafka.streams.kstream.internals.suppress.TimeDefinitions.TimeDefinition; -import org.apache.kafka.streams.processor.AbstractProcessor; -import org.apache.kafka.streams.processor.Processor; -import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorRecordContext; import org.apache.kafka.streams.processor.internals.metrics.ProcessorNodeMetrics; @@ -38,6 +35,7 @@ import static java.util.Objects.requireNonNull; +@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated. public class KTableSuppressProcessorSupplier implements KTableProcessorSupplier { private final SuppressedInternal suppress; private final String storeName; @@ -54,7 +52,7 @@ public KTableSuppressProcessorSupplier(final SuppressedInternal suppress, } @Override - public Processor> get() { + public org.apache.kafka.streams.processor.Processor> get() { return new KTableSuppressProcessor<>(suppress, storeName); } @@ -70,7 +68,7 @@ public KTableValueGetter get() { private TimeOrderedKeyValueBuffer buffer; @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { parentGetter.init(context); // the main processor is responsible for the buffer's lifecycle buffer = requireNonNull(context.getStateStore(storeName)); @@ -111,7 +109,7 @@ public boolean enableSendingOldValues(final boolean forceMaterialization) { return parentKTable.enableSendingOldValues(forceMaterialization); } - private static final class KTableSuppressProcessor extends AbstractProcessor> { + private static final class KTableSuppressProcessor extends org.apache.kafka.streams.processor.AbstractProcessor> { private final long maxRecords; private final long maxBytes; private final long suppressDurationMillis; @@ -138,7 +136,7 @@ private KTableSuppressProcessor(final SuppressedInternal suppress, final Stri @SuppressWarnings("unchecked") @Override - public void init(final ProcessorContext context) { + public void init(final org.apache.kafka.streams.processor.ProcessorContext context) { super.init(context); internalProcessorContext = (InternalProcessorContext) context; suppressionEmitSensor = ProcessorNodeMetrics.suppressionEmitSensor( diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java b/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java index c30a273641e4f..52a213d1a9050 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java @@ -22,7 +22,10 @@ * * @param the type of keys * @param the type of values + * @deprecated Since 3.0. Use {@link org.apache.kafka.streams.processor.api.Processor} or + * {@link org.apache.kafka.streams.processor.api.ContextualProcessor} instead. */ +@Deprecated public abstract class AbstractProcessor implements Processor { protected ProcessorContext context; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ConnectedStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/processor/ConnectedStoreProvider.java index 84ba1c879cc33..49c029e70b8a4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ConnectedStoreProvider.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ConnectedStoreProvider.java @@ -91,8 +91,8 @@ * } * * @see Topology#addProcessor(String, org.apache.kafka.streams.processor.api.ProcessorSupplier, String...) - * @see KStream#process(ProcessorSupplier, String...) - * @see KStream#process(ProcessorSupplier, Named, String...) + * @see KStream#process(org.apache.kafka.streams.processor.api.ProcessorSupplier, String...) + * @see KStream#process(org.apache.kafka.streams.processor.api.ProcessorSupplier, Named, String...) * @see KStream#transform(TransformerSupplier, String...) * @see KStream#transform(TransformerSupplier, Named, String...) * @see KStream#transformValues(ValueTransformerSupplier, String...) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java index 4046f2f116f8e..9d724ec1378d4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java @@ -23,7 +23,9 @@ * * @param the type of keys * @param the type of values + * @deprecated Since 3.0. Use {@link org.apache.kafka.streams.processor.api.Processor} instead. */ +@Deprecated public interface Processor { /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java index a598e72d8031b..e1d90442c6949 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java @@ -31,6 +31,7 @@ /** * Processor context interface. */ +@SuppressWarnings("deprecation") // Not deprecating the old context, since it is used by Transformers. See KAFKA-10603. public interface ProcessorContext { /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java index a3e5f30c2c464..e53a63a4215f7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java @@ -33,7 +33,9 @@ * * @param the type of keys * @param the type of values + * @deprecated Since 3.0. Use {@link org.apache.kafka.streams.processor.api.ProcessorSupplier} instead. */ +@Deprecated public interface ProcessorSupplier extends ConnectedStoreProvider, Supplier> { /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/RecordContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/RecordContext.java index f0b8ff10d3848..9b21df83dd058 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/RecordContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/RecordContext.java @@ -21,7 +21,7 @@ /** * The context associated with the current record being processed by - * an {@link Processor} + * an {@link org.apache.kafka.streams.processor.api.Processor} */ public interface RecordContext { @@ -79,8 +79,10 @@ public interface RecordContext { *

If it is triggered while processing a record streamed from the source processor, * timestamp is defined as the timestamp of the current input record; the timestamp is extracted from * {@link org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord} by {@link TimestampExtractor}. - * Note, that an upstream {@link Processor} might have set a new timestamp by calling - * {@link ProcessorContext#forward(Object, Object, To) forward(..., To.all().withTimestamp(...))}. + * Note, that an upstream {@link org.apache.kafka.streams.processor.api.Processor} + * might have set a new timestamp by calling + * {@link org.apache.kafka.streams.processor.api.ProcessorContext#forward(org.apache.kafka.streams.processor.api.Record) + * forward(..., To.all().withTimestamp(...))}. * In particular, some Kafka Streams DSL operators set result record timestamps explicitly, * to guarantee deterministic results. * diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index 103fc7cae3b49..9b392640570bc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -217,6 +217,7 @@ private static class ProcessorNodeFactory extends NodeFact this.supplier = supplier; } + @SuppressWarnings("deprecation") // Old PAPI compatibility. ProcessorNodeFactory(final String name, final String[] predecessors, final org.apache.kafka.streams.processor.ProcessorSupplier supplier) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorAdapter.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorAdapter.java index f067bbda6472d..f48bfd4b17958 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorAdapter.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorAdapter.java @@ -21,6 +21,7 @@ import org.apache.kafka.streams.processor.api.ProcessorContext; import org.apache.kafka.streams.processor.api.Record; +@SuppressWarnings("deprecation") // Old PAPI compatibility public final class ProcessorAdapter implements Processor { private final org.apache.kafka.streams.processor.Processor delegate; private InternalProcessorContext context; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index 9bf35d4f16718..dfc0b70b3511c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -62,7 +62,7 @@ public ProcessorNode(final String name, } public ProcessorNode(final String name, - final org.apache.kafka.streams.processor.Processor processor, + @SuppressWarnings("deprecation") final org.apache.kafka.streams.processor.Processor processor, final Set stateStores) { this.name = name; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java index 12df0e01dffed..bf4e5aaf9ef02 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java @@ -57,7 +57,7 @@ * .withCachingDisabled()); * } * When using the Processor API, i.e., {@link org.apache.kafka.streams.Topology Topology}, users create - * {@link StoreBuilder}s that can be attached to {@link org.apache.kafka.streams.processor.Processor Processor}s. + * {@link StoreBuilder}s that can be attached to {@link org.apache.kafka.streams.processor.api.Processor Processor}s. * For example, you can create a {@link org.apache.kafka.streams.kstream.Windowed windowed} RocksDB store with custom * changelog topic configuration like: *

{@code
diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java
index 129d29aecfa5c..fba58a1d47d17 100644
--- a/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java
@@ -76,6 +76,7 @@
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class StreamsBuilderTest {
 
     private static final String STREAM_TOPIC = "stream-topic";
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java
index 447e0a2e34ac0..4bc69e6fa1446 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java
@@ -66,6 +66,7 @@
 import static org.hamcrest.core.IsEqual.equalTo;
 import static org.junit.Assert.assertNotNull;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 @Category({IntegrationTest.class})
 public class GlobalKTableIntegrationTest {
     private static final int NUM_BROKERS = 1;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
index 5ef4345959dad..6d8932573add4 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalThreadShutDownOrderTest.java
@@ -30,7 +30,6 @@
 import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
 import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
 import org.apache.kafka.streams.kstream.Consumed;
-import org.apache.kafka.streams.processor.AbstractProcessor;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.state.KeyValueStore;
 import org.apache.kafka.streams.state.Stores;
@@ -108,6 +107,7 @@ public static void closeCluster() {
     @Rule
     public TestName testName = new TestName();
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Before
     public void before() throws Exception {
         builder = new StreamsBuilder();
@@ -196,7 +196,8 @@ private void populateTopics(final String topicName) throws Exception {
     }
 
 
-    private class GlobalStoreProcessor extends AbstractProcessor {
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+    private class GlobalStoreProcessor extends org.apache.kafka.streams.processor.AbstractProcessor {
 
         private KeyValueStore store;
         private final String storeName;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java
index df0645ed9e2e2..9deb5eb81f622 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/RestoreIntegrationTest.java
@@ -40,7 +40,6 @@
 import org.apache.kafka.streams.kstream.Consumed;
 import org.apache.kafka.streams.kstream.KStream;
 import org.apache.kafka.streams.kstream.Materialized;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.StateRestoreListener;
 import org.apache.kafka.streams.processor.TaskId;
@@ -292,6 +291,7 @@ public void shouldSuccessfullyStartWhenLoggingDisabled() throws InterruptedExcep
         assertTrue(startupLatch.await(30, TimeUnit.SECONDS));
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void shouldProcessDataFromStoresWithLoggingDisabled() throws InterruptedException {
 
@@ -430,7 +430,8 @@ static int numStoresClosed() {
         }
     }
 
-    public static class KeyValueStoreProcessor implements Processor {
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+    public static class KeyValueStoreProcessor implements org.apache.kafka.streams.processor.Processor {
 
         private final String topic;
         private final CountDownLatch processorLatch;
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java
index b332e143346c8..9c6085fec703d 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/StoreUpgradeIntegrationTest.java
@@ -27,7 +27,6 @@
 import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
 import org.apache.kafka.streams.kstream.Windowed;
 import org.apache.kafka.streams.kstream.internals.TimeWindow;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.state.KeyValueIterator;
 import org.apache.kafka.streams.state.KeyValueStore;
@@ -60,6 +59,7 @@
 import static java.util.Collections.singletonList;
 import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 @Category({IntegrationTest.class})
 public class StoreUpgradeIntegrationTest {
     private static final String STORE_NAME = "store";
@@ -953,7 +953,8 @@ private  void processKeyValueAndVerifyWindowedCountWithTimestamp(final K k
             "Could not get expected result in time.");
     }
 
-    private static class KeyValueProcessor implements Processor {
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+    private static class KeyValueProcessor implements org.apache.kafka.streams.processor.Processor {
         private KeyValueStore store;
 
         @SuppressWarnings("unchecked")
@@ -980,7 +981,8 @@ public void process(final Integer key, final Integer value) {
         public void close() {}
     }
 
-    private static class TimestampedKeyValueProcessor implements Processor {
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+    private static class TimestampedKeyValueProcessor implements org.apache.kafka.streams.processor.Processor {
         private ProcessorContext context;
         private TimestampedKeyValueStore store;
 
@@ -1013,7 +1015,8 @@ public void process(final Integer key, final Integer value) {
         public void close() {}
     }
 
-    private static class WindowedProcessor implements Processor {
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+    private static class WindowedProcessor implements org.apache.kafka.streams.processor.Processor {
         private WindowStore store;
 
         @SuppressWarnings("unchecked")
@@ -1040,7 +1043,8 @@ public void process(final Integer key, final Integer value) {
         public void close() {}
     }
 
-    private static class TimestampedWindowedProcessor implements Processor {
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+    private static class TimestampedWindowedProcessor implements org.apache.kafka.streams.processor.Processor {
         private ProcessorContext context;
         private TimestampedWindowStore store;
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java
index 9288c60443fb0..97d894f03ae29 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamsUncaughtExceptionHandlerIntegrationTest.java
@@ -31,7 +31,6 @@
 import org.apache.kafka.streams.kstream.Consumed;
 import org.apache.kafka.streams.kstream.KStream;
 import org.apache.kafka.streams.kstream.Named;
-import org.apache.kafka.streams.processor.AbstractProcessor;
 import org.apache.kafka.streams.state.Stores;
 import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder;
 import org.apache.kafka.test.IntegrationTest;
@@ -226,7 +225,7 @@ private void produceMessages(final long timestamp, final String streamOneInput,
             timestamp);
     }
 
-    private static class ShutdownProcessor extends AbstractProcessor {
+    private static class ShutdownProcessor extends org.apache.kafka.streams.processor.AbstractProcessor {
         final List valueList;
 
         ShutdownProcessor(final List valueList) {
diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/TaskMetadataIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/TaskMetadataIntegrationTest.java
index b11779211829b..b5edece70c95f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/integration/TaskMetadataIntegrationTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/integration/TaskMetadataIntegrationTest.java
@@ -27,7 +27,6 @@
 import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
 import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
 import org.apache.kafka.streams.kstream.KStream;
-import org.apache.kafka.streams.processor.AbstractProcessor;
 import org.apache.kafka.streams.processor.TaskMetadata;
 import org.apache.kafka.test.IntegrationTest;
 import org.apache.kafka.test.TestUtils;
@@ -82,6 +81,7 @@ public static void closeCluster() {
     private AtomicBoolean process;
     private AtomicBoolean commit;
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Before
     public void setup() {
         final String testId = safeUniqueTestName(getClass(), testName);
@@ -180,7 +180,8 @@ private void produceMessages(final long timestamp, final String streamOneInput,
                 timestamp);
     }
 
-    private class PauseProcessor extends AbstractProcessor {
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+    private class PauseProcessor extends org.apache.kafka.streams.processor.AbstractProcessor {
         @Override
         public void process(final String key, final String value) {
             while (!process.get()) {
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java
index 3091a6f9e24c8..5545fb6ef1a03 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/RepartitionTopicNamingTest.java
@@ -23,7 +23,6 @@
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.Topology;
 import org.apache.kafka.streams.errors.TopologyException;
-import org.apache.kafka.streams.processor.AbstractProcessor;
 import org.junit.Test;
 
 import java.time.Duration;
@@ -463,6 +462,7 @@ private int getCountOfRepartitionTopicsFound(final String topologyString, final
     }
 
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     private Topology buildTopology(final String optimizationConfig) {
         final Initializer initializer = () -> 0;
         final Aggregator aggregator = (k, v, agg) -> agg + v.length();
@@ -505,7 +505,8 @@ private Topology buildTopology(final String optimizationConfig) {
     }
 
 
-    private static class SimpleProcessor extends AbstractProcessor {
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+    private static class SimpleProcessor extends org.apache.kafka.streams.processor.AbstractProcessor {
 
         final List valueList;
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java
index 4bc7779f5ed6a..067a53129e5fc 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/AbstractStreamTest.java
@@ -30,9 +30,6 @@
 import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier;
 import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
 import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
-import org.apache.kafka.streams.processor.AbstractProcessor;
-import org.apache.kafka.streams.processor.Processor;
-import org.apache.kafka.streams.processor.ProcessorSupplier;
 import org.apache.kafka.test.MockProcessorSupplier;
 import org.junit.Test;
 
@@ -44,6 +41,7 @@
 import static org.easymock.EasyMock.verify;
 import static org.junit.Assert.assertTrue;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class AbstractStreamTest {
 
     @Test
@@ -71,6 +69,7 @@ public void testToInternalValueTransformerWithKeySupplierSuppliesNewTransformers
         verify(valueTransformerWithKeySupplier);
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testShouldBeExtensible() {
         final StreamsBuilder builder = new StreamsBuilder();
@@ -108,7 +107,7 @@ KStream randomFilter() {
         }
     }
 
-    private static class ExtendedKStreamDummy implements ProcessorSupplier {
+    private static class ExtendedKStreamDummy implements org.apache.kafka.streams.processor.ProcessorSupplier {
 
         private final Random rand;
 
@@ -117,11 +116,11 @@ private static class ExtendedKStreamDummy implements ProcessorSupplier get() {
+        public org.apache.kafka.streams.processor.Processor get() {
             return new ExtendedKStreamDummyProcessor();
         }
 
-        private class ExtendedKStreamDummyProcessor extends AbstractProcessor {
+        private class ExtendedKStreamDummyProcessor extends org.apache.kafka.streams.processor.AbstractProcessor {
             @Override
             public void process(final K key, final V value) {
                 // flip a coin and filter
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java
index a2b8029f4be69..e4d95e0d78e0d 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java
@@ -56,6 +56,7 @@ public void setUp() {
         keyValueMapper = (key, value) -> value;
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void shouldLeftJoinWithStream() {
         final MockProcessorSupplier supplier = new MockProcessorSupplier<>();
@@ -71,6 +72,7 @@ public void shouldLeftJoinWithStream() {
         verifyJoin(expected, supplier);
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void shouldInnerJoinWithStream() {
         final MockProcessorSupplier supplier = new MockProcessorSupplier<>();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java
index d3b7626c16894..b710e24b64438 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java
@@ -62,6 +62,7 @@
 import static org.junit.Assert.assertThrows;
 
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KGroupedStreamImplTest {
 
     private static final String TOPIC = "topic";
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java
index 130e299ff8835..a870cd3645ee3 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java
@@ -126,6 +126,7 @@ public void shouldNotAllowInvalidStoreNameOnReduce() {
             Materialized.as(INVALID_STORE_NAME)));
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     private MockProcessorSupplier getReducedResults(final KTable inputKTable) {
         final MockProcessorSupplier supplier = new MockProcessorSupplier<>();
         inputKTable
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java
index 1813522bcce2c..8a731f9e47453 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java
@@ -40,7 +40,7 @@ public class KStreamBranchTest {
     private final String topicName = "topic";
     private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
 
-    @SuppressWarnings({"unchecked", "deprecation"})
+    @SuppressWarnings({"unchecked", "deprecation"}) // Old PAPI. Needs to be migrated.
     @Test
     public void testKStreamBranch() {
         final StreamsBuilder builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java
index c6c2c7a7c37fb..bc3f46117eafb 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java
@@ -40,6 +40,7 @@ public class KStreamFilterTest {
 
     private final Predicate isMultipleOfThree = (key, value) -> (key % 3) == 0;
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testFilter() {
         final StreamsBuilder builder = new StreamsBuilder();
@@ -61,6 +62,7 @@ public void testFilter() {
         assertEquals(2, supplier.theCapturedProcessor().processed().size());
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testFilterNot() {
         final StreamsBuilder builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java
index e27754a301df8..a792ff668e2c8 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java
@@ -41,6 +41,7 @@
 public class KStreamFlatMapTest {
     private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String());
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testFlatMap() {
         final StreamsBuilder builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java
index a84fdcc8024eb..c1930e5c885aa 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java
@@ -41,6 +41,7 @@ public class KStreamFlatMapValuesTest {
     private final String topicName = "topic";
     private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String());
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testFlatMapValues() {
         final StreamsBuilder builder = new StreamsBuilder();
@@ -77,6 +78,7 @@ public void testFlatMapValues() {
     }
 
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testFlatMapValuesWithKeys() {
         final StreamsBuilder builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java
index d18a7a87578da..80822559d11f5 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformTest.java
@@ -20,7 +20,6 @@
 import org.apache.kafka.streams.kstream.Transformer;
 import org.apache.kafka.streams.kstream.TransformerSupplier;
 import org.apache.kafka.streams.kstream.internals.KStreamFlatTransform.KStreamFlatTransformProcessor;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.easymock.EasyMock;
 import org.easymock.EasyMockSupport;
@@ -32,6 +31,7 @@
 
 import static org.junit.Assert.assertTrue;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KStreamFlatTransformTest extends EasyMockSupport {
 
     private Number inputKey;
@@ -129,7 +129,7 @@ public void shouldGetFlatTransformProcessor() {
         EasyMock.expect(transformerSupplier.get()).andReturn(transformer);
         replayAll();
 
-        final Processor processor = processorSupplier.get();
+        final org.apache.kafka.streams.processor.Processor processor = processorSupplier.get();
 
         verifyAll();
         assertTrue(processor instanceof KStreamFlatTransformProcessor);
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java
index 36167c04c0ce3..fd6460427ca3e 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatTransformValuesTest.java
@@ -24,7 +24,6 @@
 import org.apache.kafka.streams.kstream.ValueTransformerWithKey;
 import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier;
 import org.apache.kafka.streams.kstream.internals.KStreamFlatTransformValues.KStreamFlatTransformValuesProcessor;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext;
 import org.easymock.EasyMock;
@@ -32,6 +31,7 @@
 import org.junit.Before;
 import org.junit.Test;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KStreamFlatTransformValuesTest extends EasyMockSupport {
 
     private Integer inputKey;
@@ -127,7 +127,7 @@ public void shouldGetFlatTransformValuesProcessor() {
         EasyMock.expect(valueTransformerSupplier.get()).andReturn(valueTransformer);
         replayAll();
 
-        final Processor processor = processorSupplier.get();
+        final org.apache.kafka.streams.processor.Processor processor = processorSupplier.get();
 
         verifyAll();
         assertTrue(processor instanceof KStreamFlatTransformValuesProcessor);
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java
index abebd52141973..fda87dcff544c 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableJoinTest.java
@@ -55,6 +55,7 @@ public class KStreamGlobalKTableJoinTest {
     private MockProcessor processor;
     private StreamsBuilder builder;
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Before
     public void setUp() {
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java
index 2b2541ee4ebb3..9268997ade438 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamGlobalKTableLeftJoinTest.java
@@ -55,6 +55,7 @@ public class KStreamGlobalKTableLeftJoinTest {
     private TopologyTestDriver driver;
     private StreamsBuilder builder;
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Before
     public void setUp() {
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java
index 7bdcea86a4cf5..8464ab960f097 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java
@@ -56,6 +56,7 @@
 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.ProcessorSupplier;
 import org.apache.kafka.streams.processor.internals.ProcessorTopology;
 import org.apache.kafka.streams.processor.internals.SourceNode;
 import org.apache.kafka.streams.state.KeyValueStore;
@@ -99,6 +100,7 @@
 import static org.junit.Assert.assertThrows;
 import static org.junit.Assert.assertTrue;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KStreamImplTest {
 
     private final Consumed stringConsumed = Consumed.with(Serdes.String(), Serdes.String());
@@ -2381,7 +2383,7 @@ public void shouldNotAllowNullNamedOnFlatTransformValuesWithFlatValueWithKeySupp
     public void shouldNotAllowNullProcessSupplierOnProcess() {
         final NullPointerException exception = assertThrows(
             NullPointerException.class,
-            () -> testStream.process(null));
+            () -> testStream.process((ProcessorSupplier) null));
         assertThat(exception.getMessage(), equalTo("processorSupplier can't be null"));
     }
 
@@ -2389,7 +2391,8 @@ public void shouldNotAllowNullProcessSupplierOnProcess() {
     public void shouldNotAllowNullProcessSupplierOnProcessWithStores() {
         final NullPointerException exception = assertThrows(
             NullPointerException.class,
-            () -> testStream.process(null, "storeName"));
+            () -> testStream.process((ProcessorSupplier) null,
+                                     "storeName"));
         assertThat(exception.getMessage(), equalTo("processorSupplier can't be null"));
     }
 
@@ -2397,7 +2400,8 @@ public void shouldNotAllowNullProcessSupplierOnProcessWithStores() {
     public void shouldNotAllowNullProcessSupplierOnProcessWithNamed() {
         final NullPointerException exception = assertThrows(
             NullPointerException.class,
-            () -> testStream.process(null, Named.as("processor")));
+            () -> testStream.process((ProcessorSupplier) null,
+                                     Named.as("processor")));
         assertThat(exception.getMessage(), equalTo("processorSupplier can't be null"));
     }
 
@@ -2405,7 +2409,8 @@ public void shouldNotAllowNullProcessSupplierOnProcessWithNamed() {
     public void shouldNotAllowNullProcessSupplierOnProcessWithNamedAndStores() {
         final NullPointerException exception = assertThrows(
             NullPointerException.class,
-            () -> testStream.process(null, Named.as("processor"), "stateStore"));
+            () -> testStream.process((ProcessorSupplier) null,
+                                     Named.as("processor"), "stateStore"));
         assertThat(exception.getMessage(), equalTo("processorSupplier can't be null"));
     }
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java
index 00e5b57368a32..7ce9d34b7b850 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java
@@ -62,6 +62,7 @@
 import static org.junit.Assert.assertThrows;
 import static org.junit.Assert.assertTrue;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KStreamKStreamJoinTest {
     private final static KeyValueTimestamp[] EMPTY = new KeyValueTimestamp[0];
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
index d0fb2bed71159..04fdf34a19d48 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java
@@ -52,6 +52,7 @@
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertEquals;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KStreamKStreamLeftJoinTest {
     private final static KeyValueTimestamp[] EMPTY = new KeyValueTimestamp[0];
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java
index 9c16c59c16c1d..2b809c7f2d997 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamOuterJoinTest.java
@@ -52,6 +52,7 @@
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertEquals;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KStreamKStreamOuterJoinTest {
     private final String topic1 = "topic1";
     private final String topic2 = "topic2";
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java
index c91455d3bda7d..3d338274bea33 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableJoinTest.java
@@ -66,6 +66,7 @@ public class KStreamKTableJoinTest {
     private StreamsBuilder builder;
     private final MockProcessorSupplier supplier = new MockProcessorSupplier<>();
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Before
     public void setUp() {
         builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java
index 30e0ff86e11bc..d9f227c1eb32f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java
@@ -59,6 +59,7 @@ public class KStreamKTableLeftJoinTest {
     private MockProcessor processor;
     private StreamsBuilder builder;
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Before
     public void setUp() {
         builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java
index 623dde50c0c9e..60da681df1146 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java
@@ -39,6 +39,7 @@
 public class KStreamMapTest {
     private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String());
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testMap() {
         final StreamsBuilder builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java
index 057294864babc..fe1e4a3676a20 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java
@@ -39,6 +39,7 @@ public class KStreamMapValuesTest {
     private final MockProcessorSupplier supplier = new MockProcessorSupplier<>();
     private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String());
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testFlatMapValues() {
         final StreamsBuilder builder = new StreamsBuilder();
@@ -63,6 +64,7 @@ public void testFlatMapValues() {
         assertArrayEquals(expected, supplier.theCapturedProcessor().processed().toArray());
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testMapValuesWithKeys() {
         final StreamsBuilder builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java
index a7546e900796a..d8e70f90a877b 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java
@@ -41,6 +41,7 @@ public class KStreamSelectKeyTest {
     private final String topicName = "topic_key_select";
     private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.Integer());
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testSelectKey() {
         final StreamsBuilder builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java
index 94bad59b1e51b..4ab8a45cc2d5a 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java
@@ -33,7 +33,6 @@
 import org.apache.kafka.streams.kstream.Windowed;
 import org.apache.kafka.streams.processor.StateStoreContext;
 import org.apache.kafka.streams.processor.internals.metrics.TaskMetrics;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.To;
 import org.apache.kafka.streams.processor.internals.ProcessorRecordContext;
 import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
@@ -68,6 +67,7 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KStreamSessionWindowAggregateProcessorTest {
 
     private static final long GAP_MS = 5 * 60 * 1000L;
@@ -87,7 +87,7 @@ public class KStreamSessionWindowAggregateProcessorTest {
             sessionMerger);
 
     private final List, Change>> results = new ArrayList<>();
-    private final Processor processor = sessionAggregator.get();
+    private final org.apache.kafka.streams.processor.Processor processor = sessionAggregator.get();
     private SessionStore sessionStore;
     private InternalMockProcessorContext context;
     private final Metrics metrics = new Metrics();
@@ -404,7 +404,7 @@ public void shouldLogAndMeterWhenSkippingNullKeyWithBuiltInMetrics() {
     @Test
     public void shouldLogAndMeterWhenSkippingLateRecordWithZeroGrace() {
         setup(false);
-        final Processor processor = new KStreamSessionWindowAggregate<>(
+        final org.apache.kafka.streams.processor.Processor processor = new KStreamSessionWindowAggregate<>(
             SessionWindows.with(ofMillis(10L)).grace(ofMillis(0L)),
             STORE_NAME,
             initializer,
@@ -469,7 +469,7 @@ public void shouldLogAndMeterWhenSkippingLateRecordWithZeroGrace() {
     @Test
     public void shouldLogAndMeterWhenSkippingLateRecordWithNonzeroGrace() {
         setup(false);
-        final Processor processor = new KStreamSessionWindowAggregate<>(
+        final org.apache.kafka.streams.processor.Processor processor = new KStreamSessionWindowAggregate<>(
             SessionWindows.with(ofMillis(10L)).grace(ofMillis(1L)),
             STORE_NAME,
             initializer,
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
index 0bb2c01226348..65853daeb1776 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregateTest.java
@@ -81,6 +81,7 @@
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 @RunWith(Parameterized.class)
 public class KStreamSlidingWindowAggregateTest {
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java
index 20b1d3a896676..5aad9f07fd133 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java
@@ -44,6 +44,7 @@ public class KStreamTransformTest {
     private static final String TOPIC_NAME = "topic";
     private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.Integer());
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testTransform() {
         final StreamsBuilder builder = new StreamsBuilder();
@@ -106,6 +107,7 @@ public void close() { }
         }
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testTransformWithNewDriverAndPunctuator() {
         final StreamsBuilder builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java
index 96f80ff93aadc..983e52db5c19f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java
@@ -27,7 +27,6 @@
 import org.apache.kafka.streams.kstream.ValueTransformerSupplier;
 import org.apache.kafka.streams.kstream.ValueTransformerWithKey;
 import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext;
 import org.apache.kafka.streams.TestInputTopic;
@@ -47,6 +46,7 @@
 import static org.junit.Assert.assertArrayEquals;
 
 @RunWith(EasyMockRunner.class)
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KStreamTransformValuesTest {
     private final String topicName = "topic";
     private final MockProcessorSupplier supplier = new MockProcessorSupplier<>();
@@ -54,6 +54,7 @@ public class KStreamTransformValuesTest {
     @Mock(MockType.NICE)
     private ProcessorContext context;
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testTransform() {
         final StreamsBuilder builder = new StreamsBuilder();
@@ -96,6 +97,7 @@ public void close() { }
         assertArrayEquals(expected, supplier.theCapturedProcessor().processed().toArray());
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testTransformWithKey() {
         final StreamsBuilder builder = new StreamsBuilder();
@@ -143,7 +145,7 @@ public void close() { }
     public void shouldInitializeTransformerWithForwardDisabledProcessorContext() {
         final NoOpValueTransformerWithKeySupplier transformer = new NoOpValueTransformerWithKeySupplier<>();
         final KStreamTransformValues transformValues = new KStreamTransformValues<>(transformer);
-        final Processor processor = transformValues.get();
+        final org.apache.kafka.streams.processor.Processor processor = transformValues.get();
 
         processor.init(context);
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java
index 88f3091f8a95e..39a74447dacff 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java
@@ -66,6 +66,7 @@ public class KStreamWindowAggregateTest {
     private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
     private final String threadId = Thread.currentThread().getName();
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testAggBasic() {
         final StreamsBuilder builder = new StreamsBuilder();
@@ -143,6 +144,7 @@ public void testAggBasic() {
         );
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testJoin() {
         final StreamsBuilder builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java
index b188853d27ecb..a61ed12d3a7b4 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java
@@ -49,6 +49,7 @@
 import static org.apache.kafka.common.utils.Utils.mkProperties;
 import static org.junit.Assert.assertEquals;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KTableAggregateTest {
     private final Serde stringSerde = Serdes.String();
     private final Consumed consumed = Consumed.with(stringSerde, stringSerde);
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java
index 7f772e7c693fe..d3ed6b51b1382 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java
@@ -66,6 +66,7 @@ public void setUp() {
 
     private final Predicate predicate = (key, value) -> (value % 2) == 0;
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     private void doTestKTable(final StreamsBuilder builder,
                               final KTable table2,
                               final KTable table3,
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java
index b979d3ecd7734..cac924e627453 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java
@@ -83,6 +83,7 @@ public void setUp() {
         table = new StreamsBuilder().table("test");
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testKTable() {
         final StreamsBuilder builder = new StreamsBuilder();
@@ -151,6 +152,7 @@ public void testKTable() {
             processors.get(3).processed());
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testMaterializedKTable() {
         final StreamsBuilder builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java
index 991ee5945296d..13caa9fea4494 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableInnerJoinTest.java
@@ -29,7 +29,6 @@
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.kstream.Materialized;
 import org.apache.kafka.streams.processor.MockProcessorContext;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender;
 import org.apache.kafka.streams.state.KeyValueStore;
 import org.apache.kafka.streams.test.TestRecord;
@@ -54,6 +53,7 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KTableKTableInnerJoinTest {
     private final static KeyValueTimestamp[] EMPTY = new KeyValueTimestamp[0];
 
@@ -251,7 +251,7 @@ public void shouldLogAndMeterSkippedRecordsDueToNullLeftKey() {
         final StreamsBuilder builder = new StreamsBuilder();
 
         @SuppressWarnings("unchecked")
-        final Processor> join = new KTableKTableInnerJoin<>(
+        final org.apache.kafka.streams.processor.Processor> join = new KTableKTableInnerJoin<>(
             (KTableImpl) builder.table("left", Consumed.with(Serdes.String(), Serdes.String())),
             (KTableImpl) builder.table("right", Consumed.with(Serdes.String(), Serdes.String())),
             null
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java
index 0d7f4b9444ece..0b14d8b76d4d7 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java
@@ -33,7 +33,6 @@
 import org.apache.kafka.streams.kstream.Materialized;
 import org.apache.kafka.streams.kstream.ValueMapper;
 import org.apache.kafka.streams.processor.MockProcessorContext;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender;
 import org.apache.kafka.streams.state.Stores;
 import org.apache.kafka.streams.test.TestRecord;
@@ -62,6 +61,7 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KTableKTableLeftJoinTest {
     private final String topic1 = "topic1";
     private final String topic2 = "topic2";
@@ -517,7 +517,7 @@ public void shouldLogAndMeterSkippedRecordsDueToNullLeftKey() {
         final StreamsBuilder builder = new StreamsBuilder();
 
         @SuppressWarnings("unchecked")
-        final Processor> join = new KTableKTableLeftJoin<>(
+        final org.apache.kafka.streams.processor.Processor> join = new KTableKTableLeftJoin<>(
             (KTableImpl) builder.table("left", Consumed.with(Serdes.String(), Serdes.String())),
             (KTableImpl) builder.table("right", Consumed.with(Serdes.String(), Serdes.String())),
             null
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java
index e41d654073642..6b3cf3b746a00 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java
@@ -28,7 +28,6 @@
 import org.apache.kafka.streams.kstream.Consumed;
 import org.apache.kafka.streams.kstream.KTable;
 import org.apache.kafka.streams.processor.MockProcessorContext;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender;
 import org.apache.kafka.streams.test.TestRecord;
 import org.apache.kafka.test.MockApiProcessor;
@@ -52,6 +51,7 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KTableKTableOuterJoinTest {
     private final String topic1 = "topic1";
     private final String topic2 = "topic2";
@@ -408,7 +408,7 @@ public void shouldLogAndMeterSkippedRecordsDueToNullLeftKey() {
         final StreamsBuilder builder = new StreamsBuilder();
 
         @SuppressWarnings("unchecked")
-        final Processor> join = new KTableKTableOuterJoin<>(
+        final org.apache.kafka.streams.processor.Processor> join = new KTableKTableOuterJoin<>(
                 (KTableImpl) builder.table("left", Consumed.with(Serdes.String(), Serdes.String())),
                 (KTableImpl) builder.table("right", Consumed.with(Serdes.String(), Serdes.String())),
                 null
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java
index 8e9ab437b3c2a..e8d9c6878b390 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoinTest.java
@@ -22,7 +22,6 @@
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.kstream.Consumed;
 import org.apache.kafka.streams.processor.MockProcessorContext;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender;
 import org.apache.kafka.test.StreamsTestUtils;
 import org.junit.Test;
@@ -32,6 +31,7 @@
 import static org.hamcrest.CoreMatchers.hasItem;
 import static org.hamcrest.MatcherAssert.assertThat;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KTableKTableRightJoinTest {
 
     private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
@@ -41,7 +41,7 @@ public void shouldLogAndMeterSkippedRecordsDueToNullLeftKeyWithBuiltInMetricsVer
         final StreamsBuilder builder = new StreamsBuilder();
 
         @SuppressWarnings("unchecked")
-        final Processor> join = new KTableKTableRightJoin<>(
+        final org.apache.kafka.streams.processor.Processor> join = new KTableKTableRightJoin<>(
             (KTableImpl) builder.table("left", Consumed.with(Serdes.String(), Serdes.String())),
             (KTableImpl) builder.table("right", Consumed.with(Serdes.String(), Serdes.String())),
             null
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java
index 548ac17d5622e..e761968db7ada 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java
@@ -39,6 +39,7 @@
 public class KTableMapKeysTest {
     private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.Integer(), Serdes.String());
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testMapKeysConvertingToStream() {
         final StreamsBuilder builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java
index 384d57902187d..ff6e1b996a4ed 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java
@@ -71,6 +71,7 @@ private void doTestKTable(final StreamsBuilder builder,
         }
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testKTable() {
         final StreamsBuilder builder = new StreamsBuilder();
@@ -85,6 +86,7 @@ public void testKTable() {
         doTestKTable(builder, topic1, supplier);
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testQueryableKTable() {
         final StreamsBuilder builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java
index 87d6e8745b3aa..0f5cc6b3e8b22 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableReduceTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.kafka.streams.kstream.internals;
 
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.internals.ProcessorNode;
 import org.apache.kafka.streams.state.TimestampedKeyValueStore;
 import org.apache.kafka.streams.state.ValueAndTimestamp;
@@ -32,13 +31,14 @@
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KTableReduceTest {
 
     @Test
     public void shouldAddAndSubtract() {
         final InternalMockProcessorContext>> context = new InternalMockProcessorContext<>();
 
-        final Processor>> reduceProcessor =
+        final org.apache.kafka.streams.processor.Processor>> reduceProcessor =
             new KTableReduce>(
                 "myStore",
                 this::unionNotNullArgs,
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java
index 1091d54deb4fc..e3371e059c694 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java
@@ -59,6 +59,7 @@ public class KTableSourceTest {
     private final Consumed stringConsumed = Consumed.with(Serdes.String(), Serdes.String());
     private final Properties props = StreamsTestUtils.getStreamsConfig(Serdes.String(), Serdes.String());
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     @Test
     public void testKTable() {
         final StreamsBuilder builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java
index 5ab151200c565..1690dc9423b64 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableTransformValuesTest.java
@@ -33,7 +33,6 @@
 import org.apache.kafka.streams.kstream.ValueMapper;
 import org.apache.kafka.streams.kstream.ValueTransformerWithKey;
 import org.apache.kafka.streams.kstream.ValueTransformerWithKeySupplier;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.internals.ForwardingDisabledProcessorContext;
 import org.apache.kafka.streams.processor.internals.InternalProcessorContext;
@@ -73,6 +72,7 @@
 import static org.junit.Assert.fail;
 
 @RunWith(EasyMockRunner.class)
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KTableTransformValuesTest {
     private static final String QUERYABLE_NAME = "queryable-store";
     private static final String INPUT_TOPIC = "inputTopic";
@@ -145,7 +145,7 @@ public void shouldInitializeTransformerWithForwardDisabledProcessorContext() {
         final NoOpValueTransformerWithKeySupplier transformer = new NoOpValueTransformerWithKeySupplier<>();
         final KTableTransformValues transformValues =
             new KTableTransformValues<>(parent, transformer, null);
-        final Processor> processor = transformValues.get();
+        final org.apache.kafka.streams.processor.Processor> processor = transformValues.get();
 
         processor.init(context);
 
@@ -157,7 +157,7 @@ public void shouldNotSendOldValuesByDefault() {
         final KTableTransformValues transformValues =
             new KTableTransformValues<>(parent, new ExclamationValueTransformerSupplier(), null);
 
-        final Processor> processor = transformValues.get();
+        final org.apache.kafka.streams.processor.Processor> processor = transformValues.get();
         processor.init(context);
 
         context.forward("Key", new Change<>("Key->newValue!", null));
@@ -178,7 +178,7 @@ public void shouldSendOldValuesIfConfigured() {
         replay(parent);
 
         transformValues.enableSendingOldValues(true);
-        final Processor> processor = transformValues.get();
+        final org.apache.kafka.streams.processor.Processor> processor = transformValues.get();
         processor.init(context);
 
         context.forward("Key", new Change<>("Key->newValue!", "Key->oldValue!"));
@@ -301,7 +301,7 @@ public void shouldCloseTransformerOnProcessorClose() {
         expectLastCall();
         replay(mockSupplier, transformer);
 
-        final Processor> processor = transformValues.get();
+        final org.apache.kafka.streams.processor.Processor> processor = transformValues.get();
         processor.close();
 
         verify(transformer);
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java
index d6e56ba5b6128..28dbd188884ed 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionWindowedKStreamImplTest.java
@@ -54,6 +54,7 @@
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertThrows;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class SessionWindowedKStreamImplTest {
     private static final String TOPIC = "input";
     private final StreamsBuilder builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java
index d6b26bf2a7c59..ef69c9ff68325 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SlidingWindowedKStreamImplTest.java
@@ -54,6 +54,7 @@
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertThrows;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class SlidingWindowedKStreamImplTest {
 
     private static final String TOPIC = "input";
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java
index c35da00697ba5..0a847776bd187 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowedKStreamImplTest.java
@@ -52,6 +52,7 @@
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.assertThrows;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class TimeWindowedKStreamImplTest {
     private static final String TOPIC = "input";
     private final StreamsBuilder builder = new StreamsBuilder();
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java
index 498e8edb68908..4a379d611d60d 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/foreignkeyjoin/SubscriptionResolverJoinProcessorSupplierTest.java
@@ -24,7 +24,6 @@
 import org.apache.kafka.streams.kstream.internals.KTableValueGetter;
 import org.apache.kafka.streams.kstream.internals.KTableValueGetterSupplier;
 import org.apache.kafka.streams.processor.MockProcessorContext;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.state.ValueAndTimestamp;
 import org.apache.kafka.streams.state.internals.Murmur3;
@@ -38,6 +37,7 @@
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.collection.IsEmptyCollection.empty;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class SubscriptionResolverJoinProcessorSupplierTest {
     private static final StringSerializer STRING_SERIALIZER = new StringSerializer();
     private static final ValueJoiner JOINER =
@@ -83,7 +83,7 @@ public void shouldNotForwardWhenHashDoesNotMatch() {
                 JOINER,
                 leftJoin
             );
-        final Processor> processor = processorSupplier.get();
+        final org.apache.kafka.streams.processor.Processor> processor = processorSupplier.get();
         final MockProcessorContext context = new MockProcessorContext();
         processor.init(context);
         context.setRecordMetadata("topic", 0, 0, new RecordHeaders(), 0);
@@ -108,7 +108,7 @@ public void shouldIgnoreUpdateWhenLeftHasBecomeNull() {
                 JOINER,
                 leftJoin
             );
-        final Processor> processor = processorSupplier.get();
+        final org.apache.kafka.streams.processor.Processor> processor = processorSupplier.get();
         final MockProcessorContext context = new MockProcessorContext();
         processor.init(context);
         context.setRecordMetadata("topic", 0, 0, new RecordHeaders(), 0);
@@ -133,7 +133,7 @@ public void shouldForwardWhenHashMatches() {
                 JOINER,
                 leftJoin
             );
-        final Processor> processor = processorSupplier.get();
+        final org.apache.kafka.streams.processor.Processor> processor = processorSupplier.get();
         final MockProcessorContext context = new MockProcessorContext();
         processor.init(context);
         context.setRecordMetadata("topic", 0, 0, new RecordHeaders(), 0);
@@ -159,7 +159,7 @@ public void shouldEmitTombstoneForInnerJoinWhenRightIsNull() {
                 JOINER,
                 leftJoin
             );
-        final Processor> processor = processorSupplier.get();
+        final org.apache.kafka.streams.processor.Processor> processor = processorSupplier.get();
         final MockProcessorContext context = new MockProcessorContext();
         processor.init(context);
         context.setRecordMetadata("topic", 0, 0, new RecordHeaders(), 0);
@@ -185,7 +185,7 @@ public void shouldEmitResultForLeftJoinWhenRightIsNull() {
                 JOINER,
                 leftJoin
             );
-        final Processor> processor = processorSupplier.get();
+        final org.apache.kafka.streams.processor.Processor> processor = processorSupplier.get();
         final MockProcessorContext context = new MockProcessorContext();
         processor.init(context);
         context.setRecordMetadata("topic", 0, 0, new RecordHeaders(), 0);
@@ -211,7 +211,7 @@ public void shouldEmitTombstoneForLeftJoinWhenRightIsNullAndLeftIsNull() {
                 JOINER,
                 leftJoin
             );
-        final Processor> processor = processorSupplier.get();
+        final org.apache.kafka.streams.processor.Processor> processor = processorSupplier.get();
         final MockProcessorContext context = new MockProcessorContext();
         processor.init(context);
         context.setRecordMetadata("topic", 0, 0, new RecordHeaders(), 0);
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java
index 0d75452a798fc..a898338f5fc37 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/GraphGraceSearchUtilTest.java
@@ -22,7 +22,6 @@
 import org.apache.kafka.streams.kstream.internals.KStreamSessionWindowAggregate;
 import org.apache.kafka.streams.kstream.internals.KStreamWindowAggregate;
 import org.apache.kafka.streams.kstream.internals.TimeWindow;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.state.StoreBuilder;
 import org.junit.Test;
@@ -32,6 +31,7 @@
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.fail;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class GraphGraceSearchUtilTest {
     @Test
     public void shouldThrowOnNull() {
@@ -50,7 +50,7 @@ public void shouldFailIfThereIsNoGraceAncestor() {
         final StatefulProcessorNode gracelessAncestor = new StatefulProcessorNode<>(
             "stateful",
             new ProcessorParameters<>(
-                () -> new Processor() {
+                () -> new org.apache.kafka.streams.processor.Processor() {
                     @Override
                     public void init(final ProcessorContext context) {}
 
@@ -134,7 +134,7 @@ public void shouldExtractGraceFromSessionAncestorThroughStatefulParent() {
         final StatefulProcessorNode statefulParent = new StatefulProcessorNode<>(
             "stateful",
             new ProcessorParameters<>(
-                () -> new Processor() {
+                () -> new org.apache.kafka.streams.processor.Processor() {
                     @Override
                     public void init(final ProcessorContext context) {}
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNodeTest.java
index a2c4938336376..99be7f85498ab 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNodeTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/TableProcessorNodeTest.java
@@ -17,16 +17,15 @@
 
 package org.apache.kafka.streams.kstream.internals.graph;
 
-import org.apache.kafka.streams.processor.AbstractProcessor;
-import org.apache.kafka.streams.processor.ProcessorContext;
 import org.junit.Test;
 
 import static org.junit.Assert.assertTrue;
 
 public class TableProcessorNodeTest {
-    private static class TestProcessor extends AbstractProcessor {
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+    private static class TestProcessor extends org.apache.kafka.streams.processor.AbstractProcessor {
         @Override
-        public void init(final ProcessorContext context) {
+        public void init(final org.apache.kafka.streams.processor.ProcessorContext context) {
         }
 
         @Override
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java
index fa4eada2458ea..4db1342a1e627 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorMetricsTest.java
@@ -26,7 +26,6 @@
 import org.apache.kafka.streams.kstream.Suppressed;
 import org.apache.kafka.streams.kstream.internals.Change;
 import org.apache.kafka.streams.kstream.internals.KTableImpl;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.StateStore;
 import org.apache.kafka.streams.processor.StateStoreContext;
 import org.apache.kafka.streams.processor.TaskId;
@@ -50,6 +49,7 @@
 import static org.hamcrest.Matchers.greaterThan;
 import static org.hamcrest.core.Is.is;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KTableSuppressProcessorMetricsTest {
     private static final long ARBITRARY_LONG = 5L;
     private static final TaskId TASK_ID = new TaskId(0, 0);
@@ -134,7 +134,7 @@ public void shouldRecordMetricsWithBuiltInMetricsVersionLatest() {
             .build();
 
         final KTableImpl mock = EasyMock.mock(KTableImpl.class);
-        final Processor> processor =
+        final org.apache.kafka.streams.processor.Processor> processor =
             new KTableSuppressProcessorSupplier<>(
                 (SuppressedInternal) Suppressed.untilTimeLimit(Duration.ofDays(100), maxRecords(1)),
                 storeName,
diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java
index 401996588e591..23b91c5d1c21a 100644
--- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/suppress/KTableSuppressProcessorTest.java
@@ -30,7 +30,6 @@
 import org.apache.kafka.streams.kstream.internals.SessionWindow;
 import org.apache.kafka.streams.kstream.internals.TimeWindow;
 import org.apache.kafka.streams.processor.MockProcessorContext;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.StateStore;
 import org.apache.kafka.streams.processor.StateStoreContext;
 import org.apache.kafka.streams.processor.internals.ProcessorNode;
@@ -60,13 +59,14 @@
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.junit.Assert.fail;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class KTableSuppressProcessorTest {
     private static final long ARBITRARY_LONG = 5L;
 
     private static final Change ARBITRARY_CHANGE = new Change<>(7L, 14L);
 
     private static class Harness {
-        private final Processor> processor;
+        private final org.apache.kafka.streams.processor.Processor> processor;
         private final MockInternalProcessorContext context;
 
 
@@ -81,7 +81,7 @@ private static class Harness {
                 .build();
 
             final KTableImpl parent = EasyMock.mock(KTableImpl.class);
-            final Processor> processor =
+            final org.apache.kafka.streams.processor.Processor> processor =
                 new KTableSuppressProcessorSupplier<>((SuppressedInternal) suppressed, storeName, parent).get();
 
             final MockInternalProcessorContext context = new MockInternalProcessorContext();
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java
index 9412e55e0ff60..e49f1fa8b0180 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorContextImplTest.java
@@ -21,7 +21,6 @@
 import org.apache.kafka.common.utils.Bytes;
 import org.apache.kafka.streams.StreamsConfig;
 import org.apache.kafka.streams.kstream.Windowed;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.PunctuationType;
 import org.apache.kafka.streams.processor.StateStore;
@@ -717,7 +716,7 @@ private void initStateStoreMock(final StateStore stateStore) {
     }
 
     private  void doTest(final String name, final Consumer checker) {
-        final Processor processor = new Processor() {
+        @SuppressWarnings("deprecation") final org.apache.kafka.streams.processor.Processor processor = new org.apache.kafka.streams.processor.Processor() {
             @Override
             public void init(final ProcessorContext context) {
                 final T store = context.getStateStore(name);
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java
index 33d5c4d909caa..511938da84f16 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java
@@ -25,7 +25,6 @@
 import org.apache.kafka.streams.Topology;
 import org.apache.kafka.streams.TopologyTestDriver;
 import org.apache.kafka.streams.errors.StreamsException;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.api.Record;
 import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl;
@@ -62,7 +61,8 @@ public void shouldThrowStreamsExceptionIfExceptionCaughtDuringClose() {
         assertThrows(StreamsException.class, () -> node.init(null));
     }
 
-    private static class ExceptionalProcessor implements Processor {
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+    private static class ExceptionalProcessor implements org.apache.kafka.streams.processor.Processor {
         @Override
         public void init(final ProcessorContext context) {
             throw new RuntimeException();
@@ -79,7 +79,8 @@ public void close() {
         }
     }
 
-    private static class NoOpProcessor implements Processor {
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+    private static class NoOpProcessor implements org.apache.kafka.streams.processor.Processor {
         @Override
         public void init(final ProcessorContext context) {
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
index 6f384a77b0a2a..b92a46840af35 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
@@ -32,7 +32,6 @@
 import org.apache.kafka.streams.Topology;
 import org.apache.kafka.streams.TopologyTestDriver;
 import org.apache.kafka.streams.TopologyWrapper;
-import org.apache.kafka.streams.processor.AbstractProcessor;
 import org.apache.kafka.streams.processor.StreamPartitioner;
 import org.apache.kafka.streams.processor.TimestampExtractor;
 import org.apache.kafka.streams.processor.api.Processor;
@@ -877,7 +876,8 @@ public void process(final Record record) {
     /**
      * A processor that stores each key-value pair in an in-memory key-value store registered with the context.
      */
-    protected static class OldAPIStatefulProcessor extends AbstractProcessor {
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+    protected static class OldAPIStatefulProcessor extends org.apache.kafka.streams.processor.AbstractProcessor {
         private KeyValueStore store;
         private final String storeName;
 
@@ -919,10 +919,12 @@ public void process(final Record record) {
         }
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     private  org.apache.kafka.streams.processor.ProcessorSupplier define(final org.apache.kafka.streams.processor.Processor processor) {
         return () -> processor;
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     private  org.apache.kafka.streams.processor.ProcessorSupplier defineWithStoresOldAPI(final Supplier> supplier,
                                                                                                      final Set> stores) {
         return new org.apache.kafka.streams.processor.ProcessorSupplier() {
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java
index ac56924844883..cde573a982e25 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java
@@ -16,7 +16,6 @@
  */
 package org.apache.kafka.streams.processor.internals;
 
-import org.apache.kafka.streams.processor.AbstractProcessor;
 import org.apache.kafka.streams.processor.Cancellable;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.PunctuationType;
@@ -141,7 +140,8 @@ public void punctuate(final ProcessorNode node, final long timestamp, final Punc
         assertEquals(1, node.mockProcessor.punctuatedStreamTime().size());
     }
 
-    private static class TestProcessor extends AbstractProcessor {
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+    private static class TestProcessor extends org.apache.kafka.streams.processor.AbstractProcessor {
 
         @Override
         public void init(final ProcessorContext context) {}
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java
index bd4829989a9e4..3b7eb784f7c20 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java
@@ -44,7 +44,6 @@
 import org.apache.kafka.streams.kstream.Produced;
 import org.apache.kafka.streams.kstream.Reducer;
 import org.apache.kafka.streams.kstream.StreamJoined;
-import org.apache.kafka.streams.processor.AbstractProcessor;
 import org.apache.kafka.streams.state.Stores;
 import org.apache.kafka.test.StreamsTestUtils;
 import org.junit.After;
@@ -135,6 +134,7 @@ public void shouldSendCorrectResults_NO_OPTIMIZATION() {
     }
 
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     private void runTest(final String optimizationConfig, final int expectedNumberRepartitionTopics) {
 
         final StreamsBuilder builder = new StreamsBuilder();
@@ -257,7 +257,8 @@ private List> getKeyValues() {
         return keyValueList;
     }
 
-    private static class SimpleProcessor extends AbstractProcessor {
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+    private static class SimpleProcessor extends org.apache.kafka.streams.processor.AbstractProcessor {
 
         final List valueList;
 
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
index 89846d86c0d00..b8e5d23eef908 100644
--- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
@@ -1809,6 +1809,7 @@ public void shouldNotCreateStandbyTaskIfStateStoresHaveLoggingDisabled() {
     }
 
     @Test
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     public void shouldPunctuateActiveTask() {
         final List punctuatedStreamTime = new ArrayList<>();
         final List punctuatedWallClockTime = new ArrayList<>();
@@ -1879,6 +1880,7 @@ public void process(final Object key, final Object value) {}
     }
 
     @Test
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     public void shouldPunctuateWithTimestampPreservedInProcessorContext() {
         final org.apache.kafka.streams.kstream.TransformerSupplier> punctuateProcessor =
             () -> new org.apache.kafka.streams.kstream.Transformer>() {
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java
index eb81a71800b7c..18d945855463f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/EosTestClient.java
@@ -99,6 +99,7 @@ public void start() {
         }
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     private KafkaStreams createKafkaStreams(final Properties props) {
         props.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID);
         props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1);
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java
index 2e53d580d5292..4882fdd5bdf5a 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java
@@ -160,6 +160,7 @@ private Properties getStreamsConfig(final Properties props) {
         return fullProps;
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     public Topology getTopology() {
         final StreamsBuilder builder = new StreamsBuilder();
         final Consumed stringIntConsumed = Consumed.with(stringSerde, intSerde);
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java
index e8ec04c23ca52..1222a81b2fb51 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java
@@ -23,26 +23,24 @@
 import org.apache.kafka.streams.kstream.Initializer;
 import org.apache.kafka.streams.kstream.KeyValueMapper;
 import org.apache.kafka.streams.kstream.Windowed;
-import org.apache.kafka.streams.processor.AbstractProcessor;
-import org.apache.kafka.streams.processor.Processor;
 import org.apache.kafka.streams.processor.ProcessorContext;
-import org.apache.kafka.streams.processor.ProcessorSupplier;
 
 import java.time.Instant;
 
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
 public class SmokeTestUtil {
 
     final static int END = Integer.MAX_VALUE;
 
-    static ProcessorSupplier printProcessorSupplier(final String topic) {
+    static org.apache.kafka.streams.processor.ProcessorSupplier printProcessorSupplier(final String topic) {
         return printProcessorSupplier(topic, "");
     }
 
-    static ProcessorSupplier printProcessorSupplier(final String topic, final String name) {
-        return new ProcessorSupplier() {
+    static org.apache.kafka.streams.processor.ProcessorSupplier printProcessorSupplier(final String topic, final String name) {
+        return new org.apache.kafka.streams.processor.ProcessorSupplier() {
             @Override
-            public Processor get() {
-                return new AbstractProcessor() {
+            public org.apache.kafka.streams.processor.Processor get() {
+                return new org.apache.kafka.streams.processor.AbstractProcessor() {
                     private int numRecordsProcessed = 0;
                     private long smallestOffset = Long.MAX_VALUE;
                     private long largestOffset = Long.MIN_VALUE;
diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
index 2ad07f2fa3e4a..1ea5c0f2b0c4f 100644
--- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
+++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
@@ -89,6 +89,7 @@ public static void main(final String[] args) throws Exception {
         });
     }
 
+    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     public static KafkaStreams buildStreams(final Properties streamsProperties) {
         final StreamsBuilder builder = new StreamsBuilder();
         final KStream dataStream = builder.stream("data");
diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java
index f18b763f0fe3e..a3bb87d3034dc 100644
--- a/streams/src/test/java/org/apache/kafka/test/MockProcessor.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockProcessor.java
@@ -17,7 +17,6 @@
 package org.apache.kafka.test;
 
 import org.apache.kafka.streams.KeyValueTimestamp;
-import org.apache.kafka.streams.processor.AbstractProcessor;
 import org.apache.kafka.streams.processor.Cancellable;
 import org.apache.kafka.streams.processor.ProcessorContext;
 import org.apache.kafka.streams.processor.PunctuationType;
@@ -28,7 +27,8 @@
 import java.util.List;
 import java.util.Map;
 
-public class MockProcessor extends AbstractProcessor {
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+public class MockProcessor extends org.apache.kafka.streams.processor.AbstractProcessor {
     private final MockApiProcessor delegate;
 
     public MockProcessor(final PunctuationType punctuationType,
diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java
index 7b73dd3d5b7af..c6b70f2763d82 100644
--- a/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java
+++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java
@@ -16,8 +16,6 @@
  */
 package org.apache.kafka.test;
 
-import org.apache.kafka.streams.processor.Processor;
-import org.apache.kafka.streams.processor.ProcessorSupplier;
 import org.apache.kafka.streams.processor.PunctuationType;
 
 import java.util.ArrayList;
@@ -25,7 +23,8 @@
 
 import static org.junit.Assert.assertEquals;
 
-public class MockProcessorSupplier implements ProcessorSupplier {
+@SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
+public class MockProcessorSupplier implements org.apache.kafka.streams.processor.ProcessorSupplier {
 
     private final long scheduleInterval;
     private final PunctuationType punctuationType;
@@ -45,7 +44,7 @@ public MockProcessorSupplier(final long scheduleInterval, final PunctuationType
     }
 
     @Override
-    public Processor get() {
+    public org.apache.kafka.streams.processor.Processor get() {
         final MockProcessor processor = new MockProcessor<>(punctuationType, scheduleInterval);
 
         // to keep tests simple, ignore calls from ApiUtils.checkSupplier
diff --git a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KStream.scala b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KStream.scala
index 4d63e67268718..d3f83e0e20a9e 100644
--- a/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KStream.scala
+++ b/streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KStream.scala
@@ -27,7 +27,8 @@ import org.apache.kafka.streams.kstream.{
   ValueTransformerWithKeySupplier,
   KStream => KStreamJ
 }
-import org.apache.kafka.streams.processor.{Processor, ProcessorSupplier, TopicNameExtractor}
+import org.apache.kafka.streams.processor.TopicNameExtractor
+import org.apache.kafka.streams.processor.api.ProcessorSupplier
 import org.apache.kafka.streams.scala.FunctionsCompatConversions.{
   FlatValueMapperFromFunction,
   FlatValueMapperWithKeyFromFunction,
@@ -789,11 +790,31 @@ 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: () => Processor[K, V], stateStoreNames: String*): Unit = {
-    val processorSupplierJ: ProcessorSupplier[K, V] = () => processorSupplier()
+  @deprecated(since = "3.0", message = "Use process(ProcessorSupplier, String*) instead.")
+  def process(processorSupplier: () => org.apache.kafka.streams.processor.Processor[K, V],
+              stateStoreNames: String*): Unit = {
+    val processorSupplierJ: org.apache.kafka.streams.processor.ProcessorSupplier[K, V] = () => processorSupplier()
     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`).
@@ -807,11 +828,33 @@ 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: () => Processor[K, V], named: Named, stateStoreNames: String*): Unit = {
-    val processorSupplierJ: ProcessorSupplier[K, V] = () => processorSupplier()
+  @deprecated(since = "3.0", message = "Use process(ProcessorSupplier, String*) instead.")
+  def process(processorSupplier: () => org.apache.kafka.streams.processor.Processor[K, V],
+              named: Named,
+              stateStoreNames: String*): Unit = {
+    val processorSupplierJ: org.apache.kafka.streams.processor.ProcessorSupplier[K, V] = () => processorSupplier()
     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 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 process(processorSupplier: ProcessorSupplier[K, V, Void, Void], named: Named, stateStoreNames: String*): Unit =
+    inner.process(processorSupplier, named, stateStoreNames: _*)
+
   /**
    * Group the records by their current key into a [[KGroupedStream]]
    * 

diff --git a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala index 1dc2e76561520..9653ddb1e5790 100644 --- a/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala +++ b/streams/streams-scala/src/test/scala/org/apache/kafka/streams/scala/TopologyTest.scala @@ -20,24 +20,24 @@ import java.time.Duration import java.util import java.util.{Locale, Properties} import java.util.regex.Pattern - import org.apache.kafka.common.serialization.{Serdes => SerdesJ} import org.apache.kafka.streams.kstream.{ Aggregator, Initializer, JoinWindows, KeyValueMapper, + Reducer, + Transformer, + ValueJoiner, + ValueMapper, KGroupedStream => KGroupedStreamJ, KStream => KStreamJ, KTable => KTableJ, Materialized => MaterializedJ, - Reducer, - StreamJoined => StreamJoinedJ, - Transformer, - ValueJoiner, - ValueMapper + StreamJoined => StreamJoinedJ } -import org.apache.kafka.streams.processor.{AbstractProcessor, ProcessorContext, ProcessorSupplier} +import org.apache.kafka.streams.processor.{api, ProcessorContext} +import org.apache.kafka.streams.processor.api.{Processor, ProcessorSupplier} import org.apache.kafka.streams.scala.ImplicitConversions._ import org.apache.kafka.streams.scala.serialization.{Serdes => NewSerdes} import org.apache.kafka.streams.scala.serialization.Serdes._ @@ -359,7 +359,7 @@ class TopologyTest { mappedStream .filter((k: String, _: String) => k == "B") .mapValues((v: String) => v.toUpperCase(Locale.getDefault)) - .process(() => new SimpleProcessor(processorValueCollector)) + .process(new SimpleProcessorSupplier(processorValueCollector)) val stream2 = mappedStream.groupByKey .aggregate(0)(aggregator)(Materialized.`with`(NewSerdes.stringSerde, NewSerdes.intSerde)) @@ -402,7 +402,7 @@ class TopologyTest { val reducer: Reducer[String] = (v1, v2) => v1 + ":" + v2 val valueMapper: ValueMapper[String, String] = v => v.toUpperCase(Locale.getDefault) val processorValueCollector = new util.ArrayList[String] - val processorSupplier: ProcessorSupplier[String, String] = () => new SimpleProcessor(processorValueCollector) + val processorSupplier = new SimpleProcessorSupplier(processorValueCollector) val valueJoiner2: ValueJoiner[String, Integer, String] = (value1, value2) => value1 + ":" + value2.toString val valueJoiner3: ValueJoiner[String, String, String] = (value1, value2) => value1 + ":" + value2 @@ -457,9 +457,10 @@ class TopologyTest { assertEquals(getTopologyScala.build(props).describe.toString, getTopologyJava.build(props).describe.toString) } - private class SimpleProcessor private[TopologyTest] (val valueList: util.List[String]) - extends AbstractProcessor[String, String] { - override def process(key: String, value: String): Unit = - valueList.add(value) + private class SimpleProcessorSupplier private[TopologyTest] (val valueList: util.List[String]) + extends ProcessorSupplier[String, String, Void, Void] { + + override def get(): Processor[String, String, Void, Void] = + (record: api.Record[String, String]) => valueList.add(record.value()) } } diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java index 7366e7063f994..78924b543be91 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/TopologyTestDriver.java @@ -195,7 +195,7 @@ * trigger manually via {@link #advanceWallClockTime(Duration)}. *

* Finally, when completed, make sure your tests {@link #close()} the driver to release all resources and - * {@link org.apache.kafka.streams.processor.Processor processors}. + * {@link org.apache.kafka.streams.processor.api.Processor processors}. * *

Processor state

*

diff --git a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java index 577f94f467429..0c81e3e9593b3 100644 --- a/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java +++ b/streams/test-utils/src/main/java/org/apache/kafka/streams/processor/MockProcessorContext.java @@ -56,6 +56,7 @@ * If you require more automated tests, we recommend wrapping your {@link Processor} in a minimal source-processor-sink * {@link Topology} and using the {@link TopologyTestDriver}. */ +@SuppressWarnings("deprecation") // not deprecating old PAPI Context, since it is still in use by Transformers. public class MockProcessorContext implements ProcessorContext, RecordCollector.Supplier { // Immutable fields ================================================ private final StreamsMetricsImpl metrics; diff --git a/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java b/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java index 6e3d9315266c5..e76cb4fe9a7e1 100644 --- a/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java +++ b/streams/test-utils/src/test/java/org/apache/kafka/streams/MockProcessorContextTest.java @@ -18,10 +18,8 @@ import org.apache.kafka.common.header.internals.RecordHeaders; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.MockProcessorContext; import org.apache.kafka.streams.processor.MockProcessorContext.CapturedForward; -import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.Punctuator; @@ -48,7 +46,7 @@ public class MockProcessorContextTest { @Test public void shouldCaptureOutputRecords() { - final AbstractProcessor processor = new AbstractProcessor() { + final org.apache.kafka.streams.processor.AbstractProcessor processor = new org.apache.kafka.streams.processor.AbstractProcessor() { @Override public void process(final String key, final Long value) { context().forward(key + value, key.length() + value); @@ -73,7 +71,7 @@ public void process(final String key, final Long value) { @Test public void shouldCaptureOutputRecordsUsingTo() { - final AbstractProcessor processor = new AbstractProcessor() { + final org.apache.kafka.streams.processor.AbstractProcessor processor = new org.apache.kafka.streams.processor.AbstractProcessor() { @Override public void process(final String key, final Long value) { context().forward(key + value, key.length() + value, To.all()); @@ -99,7 +97,7 @@ public void process(final String key, final Long value) { @Test public void shouldCaptureRecordsOutputToChildByName() { - final AbstractProcessor processor = new AbstractProcessor() { + final org.apache.kafka.streams.processor.AbstractProcessor processor = new org.apache.kafka.streams.processor.AbstractProcessor() { private int count = 0; @Override @@ -161,7 +159,7 @@ public void process(final String key, final Long value) { @Test public void shouldCaptureCommitsAndAllowReset() { - final AbstractProcessor processor = new AbstractProcessor() { + final org.apache.kafka.streams.processor.AbstractProcessor processor = new org.apache.kafka.streams.processor.AbstractProcessor() { private int count = 0; @Override @@ -192,7 +190,7 @@ public void process(final String key, final Long value) { @Test public void shouldStoreAndReturnStateStores() { - final AbstractProcessor processor = new AbstractProcessor() { + final org.apache.kafka.streams.processor.AbstractProcessor processor = new org.apache.kafka.streams.processor.AbstractProcessor() { @Override public void process(final String key, final Long value) { final KeyValueStore stateStore = context().getStateStore("my-state"); @@ -227,7 +225,7 @@ public void shouldCaptureApplicationAndRecordMetadata() { final Properties config = new Properties(); config.put(StreamsConfig.APPLICATION_ID_CONFIG, "testMetadata"); - final AbstractProcessor processor = new AbstractProcessor() { + final org.apache.kafka.streams.processor.AbstractProcessor processor = new org.apache.kafka.streams.processor.AbstractProcessor() { @Override public void process(final String key, final Object value) { context().forward("appId", context().applicationId()); @@ -313,7 +311,7 @@ public void process(final String key, final Object value) { @Test public void shouldCapturePunctuator() { - final Processor processor = new Processor() { + final org.apache.kafka.streams.processor.Processor processor = new org.apache.kafka.streams.processor.Processor() { @Override public void init(final ProcessorContext context) { context.schedule(