abbccdda commented on a change in pull request #9004: URL: https://github.com/apache/kafka/pull/9004#discussion_r462057744
########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java ########## @@ -764,12 +764,12 @@ private void connectProcessorAndStateStore(final String processorName, if (!sourceTopics.isEmpty()) { stateStoreNameToSourceTopics.put(stateStoreName, - Collections.unmodifiableSet(sourceTopics)); + Collections.unmodifiableSet(sourceTopics)); Review comment: format looks weird, maybe just do 4 spaces ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalProcessorContextImpl.java ########## @@ -57,12 +57,12 @@ public StateStore getStateStore(final String name) { @SuppressWarnings("unchecked") @Override - public <K, V> void forward(final K key, final V value) { - final ProcessorNode<?, ?> previousNode = currentNode(); + public <KIn, VIn> void forward(final KIn key, final VIn value) { + final ProcessorNode<?, ?, ?, ?> previousNode = currentNode(); try { - for (final ProcessorNode<?, ?> child : currentNode().children()) { + for (final ProcessorNode<?, ?, ?, ?> child : currentNode().children()) { setCurrentNode(child); - ((ProcessorNode<K, V>) child).process(key, value); + ((ProcessorNode<KIn, VIn, KIn, VIn>) child).process(key, value); // FIXME Review comment: Could we leave a more clear comment on what needs to be fixed? ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessorContext.java ########## @@ -0,0 +1,240 @@ +/* + * 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.processor.api; + +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.processor.Cancellable; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.PunctuationType; +import org.apache.kafka.streams.processor.Punctuator; +import org.apache.kafka.streams.processor.StateRestoreCallback; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.TimestampExtractor; +import org.apache.kafka.streams.processor.To; + +import java.io.File; +import java.time.Duration; +import java.util.Map; + +/** + * Processor context interface. + * + * @param <KForward> a bound on the types of keys that may be forwarded + * @param <VForward> a bound on the types of values that may be forwarded + */ +public interface ProcessorContext<KForward, VForward> { + + /** + * Returns the application id. + * + * @return the application id + */ + String applicationId(); + + /** + * Returns the task id. + * + * @return the task id + */ + TaskId taskId(); + + /** + * Returns the default key serde. + * + * @return the key serializer + */ + Serde<?> keySerde(); + + /** + * Returns the default value serde. + * + * @return the value serializer + */ + Serde<?> valueSerde(); + + /** + * Returns the state directory for the partition. + * + * @return the state directory + */ + File stateDir(); + + /** + * Returns Metrics instance. + * + * @return StreamsMetrics + */ + StreamsMetrics metrics(); + + /** + * Registers and possibly restores the specified storage engine. + * + * @param store the storage engine + * @param stateRestoreCallback the restoration callback logic for log-backed state stores upon restart + * + * @throws IllegalStateException If store gets registered after initialized is already finished + * @throws StreamsException if the store's change log does not contain the partition + */ + void register(final StateStore store, + final StateRestoreCallback stateRestoreCallback); + + /** + * Get the state store given the store name. + * + * @param name The store name + * @return The state store instance + */ + StateStore getStateStore(final String name); + + /** + * Schedules a periodic operation for processors. A processor may call this method during + * {@link Processor#init(org.apache.kafka.streams.processor.ProcessorContext) initialization} or + * {@link Processor#process(Object, Object) processing} to + * schedule a periodic callback — called a punctuation — to {@link Punctuator#punctuate(long)}. + * The type parameter controls what notion of time is used for punctuation: + * <ul> + * <li>{@link PunctuationType#STREAM_TIME} — uses "stream time", which is advanced by the processing of messages + * in accordance with the timestamp as extracted by the {@link TimestampExtractor} in use. + * The first punctuation will be triggered by the first record that is processed. + * <b>NOTE:</b> Only advanced if messages arrive</li> + * <li>{@link PunctuationType#WALL_CLOCK_TIME} — uses system time (the wall-clock time), + * which is advanced independent of whether new messages arrive. + * The first punctuation will be triggered after interval has elapsed. + * <b>NOTE:</b> This is best effort only as its granularity is limited by how long an iteration of the + * processing loop takes to complete</li> + * </ul> + * + * <b>Skipping punctuations:</b> Punctuations will not be triggered more than once at any given timestamp. + * This means that "missed" punctuation will be skipped. + * It's possible to "miss" a punctuation if: + * <ul> + * <li>with {@link PunctuationType#STREAM_TIME}, when stream time advances more than interval</li> + * <li>with {@link PunctuationType#WALL_CLOCK_TIME}, on GC pause, too short interval, ...</li> + * </ul> + * + * @param interval the time interval between punctuations (supported minimum is 1 millisecond) + * @param type one of: {@link PunctuationType#STREAM_TIME}, {@link PunctuationType#WALL_CLOCK_TIME} + * @param callback a function consuming timestamps representing the current stream or system time + * @return a handle allowing cancellation of the punctuation schedule established by this method + */ + Cancellable schedule(final Duration interval, + final PunctuationType type, + final Punctuator callback); + + /** + * Forwards a key/value pair to all downstream processors. + * Used the input record's timestamp as timestamp for the output record. + * + * @param key key + * @param value value + */ + <K extends KForward, V extends VForward> void forward(final K key, final V value); + + /** + * Forwards a key/value pair to the specified downstream processors. + * Can be used to set the timestamp of the output record. + * + * @param key key + * @param value value + * @param to the options to use when forwarding + */ + <K extends KForward, V extends VForward> void forward(final K key, final V value, final To to); + + /** + * Requests a commit. + */ + void commit(); + + /** + * Returns the topic name of the current input record; could be null if it is not + * available (for example, if this method is invoked from the punctuate call). + * + * @return the topic name + */ + String topic(); + + /** + * Returns the partition id of the current input record; could be -1 if it is not + * available (for example, if this method is invoked from the punctuate call). + * + * @return the partition id + */ + int partition(); + + /** + * Returns the offset of the current input record; could be -1 if it is not + * available (for example, if this method is invoked from the punctuate call). + * + * @return the offset + */ + long offset(); + + /** + * Returns the headers of the current input record; could be null if it is not + * available (for example, if this method is invoked from the punctuate call). + * + * @return the headers + */ + Headers headers(); + + /** + * Returns the current timestamp. + * + * <p> If it is triggered while processing a record streamed from the source processor, + * timestamp is defined as the timestamp of the current input record; the timestamp is extracted from + * {@link org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord} by {@link TimestampExtractor}. + * + * <p> If it is triggered while processing a record generated not from the source processor (for example, + * if this method is invoked from the punctuate call), timestamp is defined as the current + * task's stream time, which is defined as the largest timestamp of any record processed by the task. + * + * @return the timestamp + */ + long timestamp(); + + /** + * Returns all the application config properties as key/value pairs. + * + * <p> The config properties are defined in the {@link org.apache.kafka.streams.StreamsConfig} + * object and associated to the ProcessorContext. + * + * <p> The type of the values is dependent on the {@link org.apache.kafka.common.config.ConfigDef.Type type} of the property + * (e.g. the value of {@link org.apache.kafka.streams.StreamsConfig#DEFAULT_KEY_SERDE_CLASS_CONFIG DEFAULT_KEY_SERDE_CLASS_CONFIG} + * will be of type {@link Class}, even if it was specified as a String to + * {@link org.apache.kafka.streams.StreamsConfig#StreamsConfig(Map) StreamsConfig(Map)}). + * + * @return all the key/values from the StreamsConfig properties + */ + Map<String, Object> appConfigs(); + + /** + * Returns all the application config properties with the given key prefix, as key/value pairs + * stripping the prefix. + * + * <p> The config properties are defined in the {@link org.apache.kafka.streams.StreamsConfig} + * object and associated to the ProcessorContext. + * + * @param prefix the properties prefix + * @return the key/values matching the given prefix from the StreamsConfig properties. + */ + Map<String, Object> appConfigsWithPrefix(final String prefix); + Review comment: nit: remove extra line ########## File path: streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextAdapter.java ########## @@ -0,0 +1,230 @@ +/* + * 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.processor.internals; + +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.processor.Cancellable; +import org.apache.kafka.streams.processor.PunctuationType; +import org.apache.kafka.streams.processor.Punctuator; +import org.apache.kafka.streams.processor.StateRestoreCallback; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.To; +import org.apache.kafka.streams.processor.api.ProcessorContext; +import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; +import org.apache.kafka.streams.state.StoreBuilder; +import org.apache.kafka.streams.state.internals.ThreadCache; + +import java.io.File; +import java.time.Duration; +import java.util.Map; + +public final class ProcessorContextAdapter<KForward, VForward> + implements ProcessorContext<KForward, VForward>, InternalApiProcessorContext<KForward, VForward> { + + final InternalProcessorContext delegate; Review comment: nit: could make access private and get an accessor. ---------------------------------------------------------------- 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