bbejeck commented on code in PR #13477:
URL: https://github.com/apache/kafka/pull/13477#discussion_r1163365163


##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java:
##########
@@ -213,6 +212,30 @@ public <K, V> void send(final String topic,
                     keyClass,
                     valueClass),
                 exception);
+        } catch (final SerializationException exception) {
+            final ProducerRecord<K, V> record = new ProducerRecord<>(topic, 
partition, timestamp, key, value, headers);
+            final 
ProductionExceptionHandler.ProductionExceptionHandlerResponse response;
+            try {
+                response = 
productionExceptionHandler.onSerializationException(record, exception);
+            } catch (final Exception e) {
+                log.error("Fatal handling serialization exception on record 
{}", record, e);
+                recordSendError(topic, e, null);
+                return;
+            }
+
+            if (response == ProductionExceptionHandlerResponse.FAIL) {
+                recordSendError(topic, exception, null);

Review Comment:
   The KIP states if the response is `FAIL` streams would add a `log.error` - 
so I'd add that here.



##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java:
##########
@@ -1390,6 +1390,46 @@ public void shouldNotCloseInternalProducerForNonEOS() {
         streamsProducer.flush();
     }
 
+    @Test
+    public void 
testCollectorFlush_ThrowsStreamsExceptionUsingDefaultExceptionHandler() {
+        final ErrorStringSerializer errorSerializer = new 
ErrorStringSerializer();
+        final RecordCollector collector = new RecordCollectorImpl(
+            logContext,
+            taskId,
+            streamsProducer,
+            new DefaultProductionExceptionHandler(),
+            streamsMetrics,
+            topology
+        );
+        collector.initialize();
+        collector.send(topic, "key", "val", null, 0, null, stringSerializer, 
errorSerializer, sinkNodeName, context);
+        assertThrows(StreamsException.class, collector::flush);
+    }
+
+    @Test
+    public void 
testCollectorFlush_DoesNotThrowStreamsExceptionUsingAlwaysContinueExceptionHandler()
 {
+        final ErrorStringSerializer errorSerializer = new 
ErrorStringSerializer();
+        final RecordCollector collector = new RecordCollectorImpl(

Review Comment:
   nit: consider a method to return the `RecordCollector` and pass the 
exception hander as a parameter here and at line 1396.



##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java:
##########
@@ -213,6 +212,30 @@ public <K, V> void send(final String topic,
                     keyClass,
                     valueClass),
                 exception);
+        } catch (final SerializationException exception) {
+            final ProducerRecord<K, V> record = new ProducerRecord<>(topic, 
partition, timestamp, key, value, headers);
+            final 
ProductionExceptionHandler.ProductionExceptionHandlerResponse response;
+            try {
+                response = 
productionExceptionHandler.handleSerializationException(record, exception);
+            } catch (final Exception e) {
+                log.error("Fatal handling serialization exception on record 
{}", record, e);
+                recordSendError(topic, e, null);
+                return;
+            }
+
+            if (response == ProductionExceptionHandlerResponse.FAIL) {
+                recordSendError(topic, exception, null);
+                return;
+            }
+
+            log.info("Unable to serialize {}. Continue processing. " +

Review Comment:
   Good point. The KIP states if the response is `CONTINUE` that, streams will 
log at the `WARN` level, so I agree with your question and suggest changing it 
to `WARN`



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