vvcephei commented on a change in pull request #9004:
URL: https://github.com/apache/kafka/pull/9004#discussion_r452588767



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextReverseShim.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.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 ProcessorContextReverseShim implements 
InternalProcessorContext {
+    final InternalApiProcessorContext<Object, Object> delegate;
+
+    static InternalProcessorContext shim(final 
InternalApiProcessorContext<Object, Object> delegate) {
+        if (delegate instanceof ProcessorContextShim) {
+            return ((ProcessorContextShim<Object, Object>) delegate).delegate;

Review comment:
       You'll see this block in all the shims. There are times when the 
internal code would wind up converting new to old and then back to new. This 
block prevents us from jumping though multiple layers in that case.

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

Review comment:
       I'll have to update the KIP. Replacing ProcessorContext instead of just 
adding the generic parameters is going to avoid the Scala compatibility issue 
we faced last time.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorShim.java
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.streams.processor.api.Processor;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+
+public final class ProcessorShim<KIn, VIn, KOut, VOut> implements 
Processor<KIn, VIn, KOut, VOut> {

Review comment:
       We also need shims for the processors.

##########
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:
       Oh, just saw this again. The thing I have to fix is the assumption that 
the child's result arguments are also KIn and VIn.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextReverseShim.java
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.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 ProcessorContextReverseShim implements 
InternalProcessorContext {

Review comment:
       This is a shim converting the new internal context back to the old one, 
so that it can be injected to the old Processor instances, which need the old 
interface.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalApiProcessorContext.java
##########
@@ -0,0 +1,123 @@
+/*
+ * 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.serialization.ByteArraySerializer;
+import org.apache.kafka.common.serialization.BytesSerializer;
+import org.apache.kafka.common.utils.Bytes;
+import org.apache.kafka.streams.processor.api.ProcessorContext;
+import org.apache.kafka.streams.processor.RecordContext;
+import org.apache.kafka.streams.processor.StateStore;
+import org.apache.kafka.streams.processor.internals.Task.TaskType;
+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 
org.apache.kafka.streams.state.internals.ThreadCache.DirtyEntryFlushListener;
+
+/**
+ * For internal use so we can update the {@link RecordContext} and current
+ * {@link ProcessorNode} when we are forwarding items that have been evicted 
or flushed from
+ * {@link ThreadCache}
+ */
+public interface InternalApiProcessorContext<KForward, VForward> extends 
ProcessorContext<KForward, VForward> {

Review comment:
       This is the "InternalProcessorContext" equivalent for the new 
ProcessorContext API. It's also identical to the old one, except for the new 
arguments.
   
   Unlike the old public APIs, once I finish up the KIP implementation, we'll 
be able to delete the old InternalProcessorContext, at which point, I'll 
probably rename this class back to InternalProcessorContext.

##########
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 &mdash; called a punctuation &mdash; to 
{@link Punctuator#punctuate(long)}.
+     * The type parameter controls what notion of time is used for punctuation:
+     * <ul>
+     *   <li>{@link PunctuationType#STREAM_TIME} &mdash; 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} &mdash; 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);

Review comment:
       Since this is a new class, I've dropped the deprecated members. 
Everything else from the old ProcessorContext is preserved.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractProcessorContext.java
##########
@@ -42,7 +42,7 @@
     private final Serde<?> valueSerde;
     private boolean initialized;
     protected ProcessorRecordContext recordContext;
-    protected ProcessorNode<?, ?> currentNode;
+    protected ProcessorNode<?, ?, ?, ?> currentNode;

Review comment:
       As you're about to find out, this kind of thing is the bulk of the 
changes.
   
   Since I've only converted the ProcessorNode (an internal class) in this PR, 
you'll see the arguments right now are overwhelmingly wildcards and Object. 
ProcessorNode is almost exclusively used in the "machine room" parts of Streams 
where we don't have access to, or any benefit from, the actual generic type 
parameters of the Processors.
   
   In the follow-on PRs, where I convert the actual Processors to the new API 
is when we'll start to see the benefits.




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


Reply via email to