abbccdda commented on a change in pull request #10744: URL: https://github.com/apache/kafka/pull/10744#discussion_r640767449
########## File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java ########## @@ -124,7 +124,10 @@ private static final String TOPIC_SUFFIX = "-topic"; private static final String SINK_NAME = "KTABLE-SINK-"; - private final ProcessorSupplier<?, ?> processorSupplier; + // Temporarily setting the processorSupplier to type Object so that we can transition from the Review comment: s/transition/transit ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/To.java ########## @@ -89,4 +89,11 @@ public int hashCode() { throw new UnsupportedOperationException("To is unsafe for use in Hash collections"); } + @Override + public String toString() { + return "To{" + Review comment: nit: could we do a string format for this to read easier? ########## File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionCacheFlushListener.java ########## @@ -19,30 +19,45 @@ import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.Record; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorNode; import org.apache.kafka.streams.state.internals.CacheFlushListener; -class SessionCacheFlushListener<K, V> implements CacheFlushListener<Windowed<K>, V> { - private final InternalProcessorContext context; +class SessionCacheFlushListener<KOut, VOut> implements CacheFlushListener<Windowed<KOut>, VOut> { + private final InternalProcessorContext<Windowed<KOut>, Change<VOut>> context; + + @SuppressWarnings("rawtypes") private final ProcessorNode myNode; + @SuppressWarnings("unchecked") SessionCacheFlushListener(final ProcessorContext context) { - this.context = (InternalProcessorContext) context; + this.context = (InternalProcessorContext<Windowed<KOut>, Change<VOut>>) context; myNode = this.context.currentNode(); } @Override - public void apply(final Windowed<K> key, - final V newValue, - final V oldValue, + public void apply(final Windowed<KOut> key, + final VOut newValue, + final VOut oldValue, final long timestamp) { - final ProcessorNode prev = context.currentNode(); + @SuppressWarnings("rawtypes") final ProcessorNode prev = context.currentNode(); Review comment: Why do we put suppression inline, instead of putting it on the top of function? ########## File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableNewProcessorSupplier.java ########## @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.streams.processor.api.ProcessorSupplier; + +public interface KTableNewProcessorSupplier<KIn, VIn, KOut, VOut> extends ProcessorSupplier<KIn, Change<VIn>, KOut, Change<VOut>> { + + KTableValueGetterSupplier<KOut, VOut> view(); + + /** + * Potentially enables sending old values. + * <p> + * If {@code forceMaterialization} is {@code true}, the method will force the materialization of upstream nodes to + * enable sending old values. + * <p> + * If {@code forceMaterialization} is {@code false}, the method will only enable the sending of old values <i>if</i> + * an upstream node is already materialized. + * + * @param forceMaterialization indicates if an upstream node should be forced to materialize to enable sending old + * values. + * @return {@code true} is sending old values is enabled, i.e. either because {@code forceMaterialization} was Review comment: ...if sending old values ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java ########## @@ -24,14 +24,14 @@ import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.prepareKeySerializer; import static org.apache.kafka.streams.kstream.internals.WrappingNullableUtils.prepareValueSerializer; -public class SinkNode<KIn, VIn, KOut, VOut> extends ProcessorNode<KIn, VIn, KOut, VOut> { +public class SinkNode<KIn, VIn> extends ProcessorNode<KIn, VIn, Void, Void> { Review comment: That brought up a good question, as whether we need to override `getChild` and `getChildren` in `SinkNode` to throw as it should never be called? ########## File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableNewProcessorSupplier.java ########## @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.streams.processor.api.ProcessorSupplier; + +public interface KTableNewProcessorSupplier<KIn, VIn, KOut, VOut> extends ProcessorSupplier<KIn, Change<VIn>, KOut, Change<VOut>> { Review comment: Could we add TODO to the old interface to easily remind the potential removal work? Or we already have tickets to do it? ########## File path: streams/src/main/java/org/apache/kafka/streams/state/internals/CacheFlushListener.java ########## @@ -31,4 +34,9 @@ * @param timestamp timestamp of new value */ void apply(final K key, final V newValue, final V oldValue, final long timestamp); + + /** + * Called when records are flushed from the {@link ThreadCache} + */ + void apply(final Record<K, Change<V>> record); Review comment: Do we want to to deprecate the old apply method and use the new one? If so, could we rename one of them to `applyOld` or `applyNew` to differentiate? Additionally, we want a parameter signature for `record` -- 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. For queries about this service, please contact Infrastructure at: us...@infra.apache.org