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


##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/ProcessorParameters.java:
##########
@@ -37,30 +43,67 @@
     @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
     private final org.apache.kafka.streams.processor.ProcessorSupplier<KIn, 
VIn> oldProcessorSupplier;
     private final ProcessorSupplier<KIn, VIn, KOut, VOut> processorSupplier;
+    private final FixedKeyProcessorSupplier<KIn, VIn, VOut> 
fixedKeyProcessorSupplier;
     private final String processorName;
 
     @SuppressWarnings("deprecation") // Old PAPI compatibility.
     public ProcessorParameters(final 
org.apache.kafka.streams.processor.ProcessorSupplier<KIn, VIn> 
processorSupplier,
                                final String processorName) {
         oldProcessorSupplier = processorSupplier;
         this.processorSupplier = () -> 
ProcessorAdapter.adapt(processorSupplier.get());
+        fixedKeyProcessorSupplier = null;
         this.processorName = processorName;
     }
 
     public ProcessorParameters(final ProcessorSupplier<KIn, VIn, KOut, VOut> 
processorSupplier,
                                final String processorName) {
         oldProcessorSupplier = null;
         this.processorSupplier = processorSupplier;
+        fixedKeyProcessorSupplier = null;
+        this.processorName = processorName;
+    }
+
+    public ProcessorParameters(final FixedKeyProcessorSupplier<KIn, VIn, VOut> 
processorSupplier,
+                               final String processorName) {
+        oldProcessorSupplier = null;
+        this.processorSupplier = null;
+        fixedKeyProcessorSupplier = processorSupplier;
         this.processorName = processorName;
     }
 
     public ProcessorSupplier<KIn, VIn, KOut, VOut> processorSupplier() {
         return processorSupplier;
     }
 
-    @SuppressWarnings("deprecation") // Old PAPI. Needs to be migrated.
-    public org.apache.kafka.streams.processor.ProcessorSupplier<KIn, VIn> 
oldProcessorSupplier() {
-        return oldProcessorSupplier;
+    public FixedKeyProcessorSupplier<KIn, VIn, VOut> 
fixedKeyProcessorSupplier() {
+        return fixedKeyProcessorSupplier;
+    }
+
+    public void addProcessorTo(final InternalTopologyBuilder topologyBuilder, 
final String[] parentNodeNames) {
+        if (processorSupplier != null) {
+            topologyBuilder.addProcessor(processorName, processorSupplier, 
parentNodeNames);
+            if (processorSupplier.stores() != null) {
+                for (final StoreBuilder<?> storeBuilder : 
processorSupplier.stores()) {
+                    topologyBuilder.addStateStore(storeBuilder, processorName);
+                }
+            }
+        }
+        if (fixedKeyProcessorSupplier != null) {
+            topologyBuilder.addProcessor(processorName, 
fixedKeyProcessorSupplier, parentNodeNames);
+            if (fixedKeyProcessorSupplier.stores() != null) {
+                for (final StoreBuilder<?> storeBuilder : 
fixedKeyProcessorSupplier.stores()) {
+                    topologyBuilder.addStateStore(storeBuilder, processorName);
+                }
+            }
+        }
+
+        // temporary hack until KIP-478 is fully implemented
+        // Old PAPI. Needs to be migrated.
+        if (oldProcessorSupplier != null && oldProcessorSupplier.stores() != 
null) {
+            for (final StoreBuilder<?> storeBuilder : 
oldProcessorSupplier.stores()) {
+                topologyBuilder.addStateStore(storeBuilder, processorName);
+            }
+        }

Review Comment:
   Interestingly enough, it seems that the old processor is not being added to 
the topology builder anymore. 
   Internal Topology doesn't even have a `#addProcessor(String, 
OldProcessorSupplier, ...)` method anymore.
   Do you think there might be some regression here?



-- 
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