mjsax commented on code in PR #20910:
URL: https://github.com/apache/kafka/pull/20910#discussion_r2536832694


##########
streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java:
##########
@@ -497,6 +522,218 @@ private <VR> KTable<K, VR> doTransformValues(final 
ValueTransformerWithKeySuppli
             builder);
     }
 
+    private <VR> FixedKeyProcessorSupplier<? super K, ? super V, ? extends VR> 
createValueTransformerWithKeySupplierToFixedProcessorSupplierAdaptor(
+        final ValueTransformerWithKeySupplier<? super K, ? super V, ? extends 
VR> transformerSupplier
+    ) {
+        return new FixedKeyProcessorSupplier<>() {
+
+            @Override
+            public Set<StoreBuilder<?>> stores() {
+                return transformerSupplier.stores();
+            }
+
+            @Override
+            public FixedKeyProcessor<K, V, VR> get() {
+                final ValueTransformerWithKey<? super K, ? super V, ? extends 
VR> valueTransformerWithKey = transformerSupplier.get();
+
+                return new FixedKeyProcessor<>() {
+                    private final AtomicReference<Long> timestamp = new 
AtomicReference<>();
+                    private final AtomicReference<Headers> headers = new 
AtomicReference<>();
+
+                    private FixedKeyProcessorContext<K, VR> context;
+
+                    @Override
+                    public void init(final FixedKeyProcessorContext<K, VR> 
context) {
+                        this.context = context;
+
+                        timestamp.set(-1L);
+                        headers.set(new RecordHeaders());
+
+                        valueTransformerWithKey.init(new ProcessorContext() {
+                            @Override
+                            public String applicationId() {
+                                return context.applicationId();
+                            }
+
+                            @Override
+                            public TaskId taskId() {
+                                return context.taskId();
+                            }
+
+                            @Override
+                            public Serde<?> keySerde() {
+                                return context.keySerde();
+                            }
+
+                            @Override
+                            public Serde<?> valueSerde() {
+                                return context.valueSerde();
+                            }
+
+                            @Override
+                            public File stateDir() {
+                                return context.stateDir();
+                            }
+
+                            @Override
+                            public StreamsMetrics metrics() {
+                                return context.metrics();
+                            }
+
+                            @Override
+                            public void register(final StateStore store, final 
StateRestoreCallback stateRestoreCallback) {
+                                ((InternalProcessorContext<?, ?>) 
context).register(store, stateRestoreCallback);
+                            }
+
+                            @Override
+                            public <S extends StateStore> S 
getStateStore(final String name) {
+                                return context.getStateStore(name);
+                            }
+
+                            @Override
+                            public Cancellable schedule(final Duration 
interval, final PunctuationType type, final Punctuator callback) {
+                                return context.schedule(interval, type, 
callback);
+                            }
+
+                            @Override
+                            public Cancellable schedule(final Instant 
startTime, final Duration interval, final PunctuationType type, final 
Punctuator callback) {
+                                return context.schedule(startTime, interval, 
type, callback);
+                            }
+
+                            @Override
+                            public <K, V> void forward(final K key, final V 
value) {
+                                throw new StreamsException("Disabled");

Review Comment:
   Cf deleted `ForwardingDisabledProcessorContext`. (same below)



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to