apoorvmittal10 commented on code in PR #16093:
URL: https://github.com/apache/kafka/pull/16093#discussion_r1617612390


##########
streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java:
##########
@@ -926,6 +933,11 @@ public class StreamsConfig extends AbstractConfig {
                     DefaultProductionExceptionHandler.class.getName(),
                     Importance.MEDIUM,
                     DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_DOC)
+            .define(PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG,
+                Type.CLASS,
+                ProcessingLogAndFailExceptionHandler.class.getName(),
+                Importance.MEDIUM,
+                PROCESSING_EXCEPTION_HANDLER_CLASS_DOC)

Review Comment:
   nit: seems the indentation is misaligned.



##########
streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java:
##########
@@ -1915,6 +1927,11 @@ public ProductionExceptionHandler 
defaultProductionExceptionHandler() {
         return 
getConfiguredInstance(DEFAULT_PRODUCTION_EXCEPTION_HANDLER_CLASS_CONFIG, 
ProductionExceptionHandler.class);
     }
 
+    @SuppressWarnings("WeakerAccess")

Review Comment:
   Query: why do we require to suppression here? I don't think there should 
generate any warnings with this method.



##########
streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContextImpl.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.errors;
+
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.streams.processor.TaskId;
+
+public class ErrorHandlerContextImpl implements ErrorHandlerContext {
+    private final String topic;
+    private final int partition;
+    private final long offset;
+    private final Headers headers;
+    private final byte[] sourceRawKey;
+    private final byte[] sourceRawValue;
+    private final String processorNodeId;
+    private final TaskId taskId;
+
+    public ErrorHandlerContextImpl(final String topic,
+        final int partition,
+        final long offset,
+        final Headers headers,
+        final byte[] sourceRawKey,
+        final byte[] sourceRawValue,
+        final String processorNodeId,
+        final TaskId taskId) {

Review Comment:
   nit: may be a line break to make it more readable/differentiate input params 
and assignment.
   
   ```suggestion
           final TaskId taskId
      ) {
   ```



##########
streams/src/main/java/org/apache/kafka/streams/errors/ErrorHandlerContextImpl.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.errors;
+
+import org.apache.kafka.common.header.Headers;
+import org.apache.kafka.streams.processor.TaskId;
+
+public class ErrorHandlerContextImpl implements ErrorHandlerContext {

Review Comment:
   May be add class javadoc please.



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java:
##########
@@ -288,7 +305,38 @@ private <K, V> void forwardInternal(final ProcessorNode<K, 
V, ?, ?> child,
                                         final Record<K, V> record) {
         setCurrentNode(child);
 
-        child.process(record);
+        try {
+            child.process(record);
+        } catch (final StreamsException e) {
+            // exception received from child processor, just rethrow
+            throw e;
+        } catch (final Exception e) {
+            final byte[] rawKey = streamTask.rawRecord() != null ? 
streamTask.rawRecord().key() : null;
+            final byte[] rawValue = streamTask.rawRecord() != null ? 
streamTask.rawRecord().value() : null;
+
+            final ErrorHandlerContext errorHandlerContext = new 
ErrorHandlerContextImpl(topic(),
+                    partition(), offset(), headers(), rawKey, rawValue,
+                    child.name(), taskId());
+            final ProcessingExceptionHandler.ProcessingHandlerResponse 
response = processingExceptionHandler
+                    .handle(errorHandlerContext, record, e);
+
+            if (response == 
ProcessingExceptionHandler.ProcessingHandlerResponse.FAIL) {
+                throw new StreamsException("Processing exception handler is 
set to fail upon" +
+                        " a processing error. If you would rather have the 
streaming pipeline" +
+                        " continue after a processing error, please set the " +
+                        PROCESSING_EXCEPTION_HANDLER_CLASS_CONFIG + " 
appropriately.",
+                        e);
+            } else {
+                log.warn(
+                    "Skipping record due to processing error. topic=[{}] 
partition=[{}] offset=[{}]",
+                    topic(),
+                    partition(),
+                    offset(),
+                    e
+                );

Review Comment:
   For `ProcessingLogAndContinueExceptionHandler` seems we will generate 2 
`warn` log messages with similar data. Do we see any additional value in this 
second log?



-- 
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: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to