jeqo commented on code in PR #11993:
URL: https://github.com/apache/kafka/pull/11993#discussion_r848644551


##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java:
##########
@@ -1501,9 +1508,60 @@ public void process(final 
org.apache.kafka.streams.processor.ProcessorSupplier<?
     }
 
     @Override
-    public void process(final ProcessorSupplier<? super K, ? super V, Void, 
Void> processorSupplier,
-                        final Named named,
-                        final String... stateStoreNames) {
+    public <KOut, VOut> KStream<KOut, VOut> process(
+        final ProcessorSupplier<? super K, ? super V, KOut, VOut> 
processorSupplier,
+        final String... stateStoreNames
+    ) {
+        return process(processorSupplier, 
Named.as(builder.newProcessorName(PROCESSOR_NAME)), stateStoreNames);
+    }
+
+    @Override
+    public <KOut, VOut> KStream<KOut, VOut> process(
+        final ProcessorSupplier<? super K, ? super V, KOut, VOut> 
processorSupplier,
+        final Named named,
+        final String... stateStoreNames
+    ) {
+        Objects.requireNonNull(processorSupplier, "processorSupplier can't be 
null");
+        Objects.requireNonNull(named, "named can't be null");
+        Objects.requireNonNull(stateStoreNames, "stateStoreNames can't be a 
null array");
+        ApiUtils.checkSupplier(processorSupplier);
+        for (final String stateStoreName : stateStoreNames) {
+            Objects.requireNonNull(stateStoreName, "stateStoreNames can't be 
null");
+        }
+
+        final String name = new NamedInternal(named).name();
+        final StatefulProcessorNode<? super K, ? super V> processNode = new 
StatefulProcessorNode<>(
+            name,
+            new ProcessorParameters<>(processorSupplier, name),
+            stateStoreNames);
+
+        builder.addGraphNode(graphNode, processNode);
+
+        // cannot inherit key and value serde
+        return new KStreamImpl<>(
+            name,
+            null,
+            null,
+            subTopologySourceNodes,
+            true,
+            processNode,
+            builder);
+    }
+
+    @Override
+    public <VOut> KStream<K, VOut> processValues(
+        final FixedKeyProcessorSupplier<? super K, ? super V, VOut> 
processorSupplier,
+        final String... stateStoreNames
+    ) {
+        return processValues(processorSupplier, NamedInternal.empty(), 
stateStoreNames);

Review Comment:
   Great catch! Do you think a new processor name is needed for 
`processValues`? e.g. `KSTREAM-PROCESSVALUES-`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to