...
Code Block |
---|
public synchronized <KIn, VIn, KOut, VOut> Topology addProcessor( final String name, final processor.api.ProcessorSupplier<KIn, VIn, KOut, VOut> supplier, final String... parentNames ); public synchronized <KIn, VIn, KOut, VOut> Topology addGlobalStore( final StoreBuilder storeBuilder, final String sourceName, final Deserializer<KIn> keyDeserializer, final Deserializer<VIn> valueDeserializer, final String topic, final String processorName, final processor.api.ProcessorSupplier<KIn, VIn, KOut, VOut> stateUpdateSupplier, ); public synchronized <KIn, VIn, KOut, VOut> Topology addGlobalStore( final StoreBuilder storeBuilder, final String sourceName, final TimestampExtractor timestampExtractor, final Deserializer<KIn> keyDeserializer, final Deserializer<VIn> valueDeserializer, final String topic, final String processorName, final processor.api.ProcessorSupplier<KIn, VIn, KOut, VOut> stateUpdateSupplier, ); |
(
...
deprecation and new method) org.apache.kafka.streams.kstream.KStream.process
The "process" method will continue to accept the old Processor<K,V> interface. Replacing this method with a new Processor<KIn, VIn, KOut, VOut> will better be done in a follow-on KIP that also proposes a way to update/replace the Transformer operations.
This future work is tracked as Note that this API is a candidate for change in the future as a part of
Jira | ||||||
---|---|---|---|---|---|---|
|
In the mean time, we will provide a migration path to the new PAPI. Since the KStreams.process currently does not allow forwarding, we will set the KOut and VOut parameters to Void, Void.
Code Block |
---|
// DEPRECATIONS:
/*
...
* @deprecated Since 3.0. Use {@link KStream#process(org.apache.kafka.streams.processor.api.ProcessorSupplier, java.lang.String...)} instead.
*/
@Deprecated
void process(
org.apache.kafka.streams.processor.ProcessorSupplier<? super K, ? super V> processorSupplier,
final String... stateStoreNames
);
*/
...
* @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(
org.apache.kafka.streams.processor.ProcessorSupplier<? super K, ? super V> processorSupplier,
Named named,
String... stateStoreNames
);
// NEW METHODS:
void process(
ProcessorSupplier<? super K, ? super V, Void, Void> processorSupplier,
String... stateStoreNames
);
void process(
ProcessorSupplier<? super K, ? super V, Void, Void> processorSupplier,
Named named,
String... stateStoreNames
); |
We will also do the same with the Scala API. Note that we depart from the typical scala-api pattern for suppliers (`()=>Processor`) and take a ProcessorSupplier, because otherwise the new and old methods will clash after type erasure.
Also, we are taking the forwarding type as Void instead of Unit because it is not possible for the scala API implementation to convert a `ProcessorSupplier[K, V, Unit, Unit]` parameter to a `ProcessorSupplier[K, V, Void, Void]` argument to the java API. The only impact of this is that implementers would have to call forward with `forward(null, null)` instead of `forward((),())`. Since the actual intent is for implementers not to call forward at all, this seems like an inconsequential incongruity.
Code Block |
---|
// DEPRECATIONS:
@deprecated(since = "3.0", message = "Use process(ProcessorSupplier, String*) instead.")
def process(
processorSupplier: () => org.apache.kafka.streams.processor.Processor[K, V],
stateStoreNames: String*
): Unit
@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
// NEW METHODS
def process(processorSupplier: ProcessorSupplier[K, V, Void, Void], stateStoreNames: String*): Unit
def process(processorSupplier: ProcessorSupplier[K, V, Void, Void], named: Named, stateStoreNames: String*): Unit |
(unchanged) org.apache.kafka.streams.kstream.{Transformer, ValueTransformer, ValueTransformerWithKey}
...