guozhangwang commented on a change in pull request #9361:
URL: https://github.com/apache/kafka/pull/9361#discussion_r498560275



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/api/Record.java
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.header.internals.RecordHeaders;
+import org.apache.kafka.streams.errors.StreamsException;
+
+/**
+ * A data class representing an incoming record for processing in a {@link 
Processor}
+ * or a record to forward to downstream processors via {@link 
ProcessorContext}.
+ *
+ * This class encapsulates all the data attributes of a record: the key and 
value, but
+ * also the timestamp of the record and any record headers.
+ *
+ * This class is immutable, though the objects referenced in the attributes of 
this class
+ * may themselves be mutable.

Review comment:
       nit: see my other comment above, maybe we can leave some guidance on 
when it is preferable to reuse the Record with mutable fields than creating a 
new Record object?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
##########
@@ -159,84 +157,123 @@ public StateStore getStateStore(final String name) {
         }
 
         final StateStore store = stateManager.getStore(name);
-        return getReadWriteStore(store);
+        return (S) getReadWriteStore(store);
     }
 
     @Override
     public <K, V> void forward(final K key,
                                final V value) {
-        throwUnsupportedOperationExceptionIfStandby("forward");
-        forward(key, value, SEND_TO_ALL);
+        final Record<K, V> toForward = new Record<>(
+            key,
+            value,
+            timestamp(),
+            headers()
+        );
+        forward(toForward);
     }
 
     @Override
     @Deprecated
     public <K, V> void forward(final K key,
                                final V value,
                                final int childIndex) {
-        throwUnsupportedOperationExceptionIfStandby("forward");
-        forward(
+        final Record<K, V> toForward = new Record<>(
             key,
             value,
-            To.child((currentNode().children()).get(childIndex).name()));
+            timestamp(),
+            headers()
+        );
+        forward(toForward, (currentNode().children()).get(childIndex).name());
     }
 
     @Override
     @Deprecated
     public <K, V> void forward(final K key,
                                final V value,
                                final String childName) {
-        throwUnsupportedOperationExceptionIfStandby("forward");
-        forward(key, value, To.child(childName));
+        final Record<K, V> toForward = new Record<>(
+            key,
+            value,
+            timestamp(),
+            headers()
+        );
+        forward(toForward, childName);
     }
 
-    @SuppressWarnings("unchecked")
     @Override
     public <K, V> void forward(final K key,
                                final V value,
                                final To to) {
+        final ToInternal toInternal = new ToInternal(to);
+        final Record<K, V> toForward = new Record<>(
+            key,
+            value,
+            toInternal.hasTimestamp() ? toInternal.timestamp() : timestamp(),
+            headers()
+        );
+        forward(toForward, toInternal.child());
+    }
+
+    @Override
+    public <K, V> void forward(final Record<K, V> record) {
+        forward(record, null);
+    }
+
+    @SuppressWarnings("unchecked")

Review comment:
       I was hoping that with the new strong typing API, during forwarding we 
do not need to cast to `(ProcessorNode<K, V, ?, ?>)` and not need this 
suppression any more.. Could we add the typing into `currentNode` (e.g. 
validate that the currentNode.children is indeed in `K, V`) instead of force 
casting?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/api/Record.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.header.internals.RecordHeaders;
+import org.apache.kafka.streams.errors.StreamsException;
+
+/**
+ * A data class representing an incoming record for processing in a {@link 
Processor}
+ * or a record to forward to downstream processors via {@link 
ProcessorContext}.
+ *
+ * This class encapsulates all the data attributes of a record: the key and 
value, but
+ * also the timestamp of the record and any record headers.
+ *
+ * This class is immutable, though the objects referenced in the attributes of 
this class
+ * may themselves be mutable.
+ *
+ * @param <K> The type of the key
+ * @param <V> The type of the value
+ */
+public class Record<K, V> {
+    private final K key;
+    private final V value;
+    private final long timestamp;
+    private final Headers headers;
+
+    /**
+     * The full constructor, specifying all the attributes of the record.
+     *
+     * @param key The key of the record. May be null.
+     * @param value The value of the record. May be null.
+     * @param timestamp The timestamp of the record. May not be negative.
+     * @param headers The headers of the record. May be null, which will cause 
subsequent calls
+     *                to {@link this#headers()} to return a non-null, empty, 
{@link Headers} collection.
+     *
+     * @throws IllegalArgumentException if the timestamp is negative.
+     */
+    public Record(final K key, final V value, final long timestamp, final 
Headers headers) {
+        this.key = key;
+        this.value = value;
+        if (timestamp < 0) {
+            throw new StreamsException(
+                "Malformed Record",
+                new IllegalArgumentException("Timestamp may not be negative. 
Got: " + timestamp)
+            );
+        }
+        this.timestamp = timestamp;
+        this.headers = new RecordHeaders(headers);

Review comment:
       That looks reasonable to me. More generally I'd suggest to document 
either in this class or in the `forward` class what side-effects the user need 
to consider if they decided to reuse the object passed in as parameters and 
mutate its fields / forward to downstream. And as long as we do that I feel 
this would be okay.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/api/ProcessorContext.java
##########
@@ -140,76 +138,25 @@ Cancellable schedule(final Duration interval,
                          final Punctuator callback);
 
     /**
-     * Forwards a key/value pair to all downstream processors.
-     * Used the input record's timestamp as timestamp for the output record.
+     * Forwards a record to all child processors.
      *
-     * @param key key
-     * @param value value
+     * @param record The record to forward to all children
      */
-    <K extends KForward, V extends VForward> void forward(final K key, final V 
value);
+    <K extends KForward, V extends VForward> void forward(Record<K, V> record);

Review comment:
       I'd suggest we add a few more sentences about "always create a new 
Record upon forwarding" v.s. "reuse the object overriding its 
key/value/timestamp/header fields", e.g. which way is more plausible when.

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
##########
@@ -135,16 +132,17 @@ public void logChange(final String storeName,
      * @throws StreamsException if an attempt is made to access this state 
store from an unknown node
      * @throws UnsupportedOperationException if the current streamTask type is 
standby
      */
+    @SuppressWarnings("unchecked")

Review comment:
       Why we'd need to add this suppression?




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


Reply via email to